From 4726cdd3dd9eabb8f923ce537b42a3d4ec6c16b8 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 8 Nov 2023 10:08:52 +0800 Subject: [PATCH 01/77] fix(stream agg): use executor schema derived by optimizer instead of generating again in agg executors (#13289) Signed-off-by: Richard Chien --- src/stream/src/executor/agg_common.rs | 7 ++- src/stream/src/executor/aggregation/mod.rs | 27 ---------- src/stream/src/executor/hash_agg.rs | 13 +---- src/stream/src/executor/integration_tests.rs | 21 +++++--- src/stream/src/executor/simple_agg.rs | 9 +--- .../src/executor/stateless_simple_agg.rs | 31 ++++++------ src/stream/src/executor/test_utils.rs | 50 ++++++++++++++++--- src/stream/src/from_proto/hash_agg.rs | 7 ++- src/stream/src/from_proto/simple_agg.rs | 7 ++- .../src/from_proto/stateless_simple_agg.rs | 7 ++- 10 files changed, 96 insertions(+), 83 deletions(-) diff --git a/src/stream/src/executor/agg_common.rs b/src/stream/src/executor/agg_common.rs index d1ea23068d430..b1feac670d942 100644 --- a/src/stream/src/executor/agg_common.rs +++ b/src/stream/src/executor/agg_common.rs @@ -20,10 +20,10 @@ use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::aggregation::AggStateStorage; -use super::Executor; +use super::{Executor, ExecutorInfo}; use crate::common::table::state_table::StateTable; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ActorContextRef, PkIndices}; +use crate::executor::ActorContextRef; use crate::task::AtomicU64Ref; /// Arguments needed to construct an `XxxAggExecutor`. @@ -33,8 +33,7 @@ pub struct AggExecutorArgs { // basic pub input: Box, pub actor_ctx: ActorContextRef, - pub pk_indices: PkIndices, - pub executor_id: u64, + pub info: ExecutorInfo, // system configs pub extreme_cache_size: usize, diff --git a/src/stream/src/executor/aggregation/mod.rs b/src/stream/src/executor/aggregation/mod.rs index 9bb1113152962..ac4826a3f1c8d 100644 --- a/src/stream/src/executor/aggregation/mod.rs +++ b/src/stream/src/executor/aggregation/mod.rs @@ -19,14 +19,12 @@ use risingwave_common::array::ArrayImpl::Bool; use risingwave_common::array::DataChunk; use risingwave_common::bail; use risingwave_common::buffer::Bitmap; -use risingwave_common::catalog::{Field, Schema}; use risingwave_expr::aggregate::{AggCall, AggKind}; use risingwave_expr::expr::{LogReport, NonStrictExpression}; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; use crate::executor::error::StreamExecutorResult; -use crate::executor::Executor; mod agg_group; mod agg_state; @@ -34,31 +32,6 @@ mod agg_state_cache; mod distinct; mod minput; -/// Generate [`crate::executor::HashAggExecutor`]'s schema from `input`, `agg_calls` and -/// `group_key_indices`. For [`crate::executor::HashAggExecutor`], the group key indices should -/// be provided. -pub fn generate_agg_schema( - input: &dyn Executor, - agg_calls: &[AggCall], - group_key_indices: Option<&[usize]>, -) -> Schema { - let aggs = agg_calls - .iter() - .map(|agg| Field::unnamed(agg.return_type.clone())); - - let fields = if let Some(key_indices) = group_key_indices { - let keys = key_indices - .iter() - .map(|idx| input.schema().fields[*idx].clone()); - - keys.chain(aggs).collect() - } else { - aggs.collect() - }; - - Schema { fields } -} - pub async fn agg_call_filter_res( agg_call: &AggCall, chunk: &DataChunk, diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 2a321dcc2b64c..9bdfc48047338 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -47,7 +47,7 @@ use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::StateTable; use crate::common::StreamChunkBuilder; use crate::error::StreamResult; -use crate::executor::aggregation::{generate_agg_schema, AggGroup as GenericAggGroup}; +use crate::executor::aggregation::AggGroup as GenericAggGroup; use crate::executor::error::StreamExecutorError; use crate::executor::monitor::StreamingMetrics; use crate::executor::{BoxedMessageStream, Executor, Message}; @@ -217,11 +217,6 @@ impl Executor for HashAggExecutor { impl HashAggExecutor { pub fn new(args: AggExecutorArgs) -> StreamResult { let input_info = args.input.info(); - let schema = generate_agg_schema( - args.input.as_ref(), - &args.agg_calls, - Some(&args.extra.group_key_indices), - ); let group_key_len = args.extra.group_key_indices.len(); // NOTE: we assume the prefix of table pk is exactly the group key @@ -239,11 +234,7 @@ impl HashAggExecutor { _phantom: PhantomData, version: args.version, actor_ctx: args.actor_ctx, - info: ExecutorInfo { - schema, - pk_indices: args.pk_indices, - identity: format!("HashAggExecutor {:X}", args.executor_id), - }, + info: args.info, input_pk_indices: input_info.pk_indices, input_schema: input_info.schema, group_key_indices: args.extra.group_key_indices, diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index cd505093294f1..7346241fccab9 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -31,7 +31,9 @@ use crate::executor::dispatch::*; use crate::executor::exchange::output::{BoxedOutput, LocalOutput}; use crate::executor::monitor::StreamingMetrics; use crate::executor::receiver::ReceiverExecutor; -use crate::executor::test_utils::agg_executor::new_boxed_simple_agg_executor; +use crate::executor::test_utils::agg_executor::{ + generate_agg_schema, new_boxed_simple_agg_executor, +}; use crate::executor::{Executor, MergeExecutor, ProjectExecutor, StatelessSimpleAggExecutor}; use crate::task::SharedContext; @@ -47,16 +49,21 @@ async fn test_merger_sum_aggr() { fields: vec![Field::unnamed(DataType::Int64)], }; let input = ReceiverExecutor::for_test(input_rx); + let agg_calls = vec![ + AggCall::from_pretty("(count:int8)"), + AggCall::from_pretty("(sum:int8 $0:int8)"), + ]; + let schema = generate_agg_schema(&input, &agg_calls, None); // for the local aggregator, we need two states: row count and sum let aggregator = StatelessSimpleAggExecutor::new( actor_ctx.clone(), input.boxed(), - vec![ - AggCall::from_pretty("(count:int8)"), - AggCall::from_pretty("(sum:int8 $0:int8)"), - ], - vec![], - 1, + ExecutorInfo { + schema, + pk_indices: vec![], + identity: format!("StatelessSimpleAggExecutor {:X}", 1), + }, + agg_calls, ) .unwrap(); let (tx, rx) = channel_for_test(); diff --git a/src/stream/src/executor/simple_agg.rs b/src/stream/src/executor/simple_agg.rs index 92730218ca148..081b66a04bedc 100644 --- a/src/stream/src/executor/simple_agg.rs +++ b/src/stream/src/executor/simple_agg.rs @@ -29,7 +29,7 @@ use super::monitor::StreamingMetrics; use super::*; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; -use crate::executor::aggregation::{generate_agg_schema, AggGroup}; +use crate::executor::aggregation::AggGroup; use crate::executor::error::StreamExecutorError; use crate::executor::{BoxedMessageStream, Message}; use crate::task::AtomicU64Ref; @@ -135,17 +135,12 @@ impl Executor for SimpleAggExecutor { impl SimpleAggExecutor { pub fn new(args: AggExecutorArgs) -> StreamResult { let input_info = args.input.info(); - let schema = generate_agg_schema(args.input.as_ref(), &args.agg_calls, None); Ok(Self { input: args.input, inner: ExecutorInner { version: args.version, actor_ctx: args.actor_ctx, - info: ExecutorInfo { - schema, - pk_indices: args.pk_indices, - identity: format!("SimpleAggExecutor-{:X}", args.executor_id), - }, + info: args.info, input_pk_indices: input_info.pk_indices, input_schema: input_info.schema, agg_funcs: args.agg_calls.iter().map(build_retractable).try_collect()?, diff --git a/src/stream/src/executor/stateless_simple_agg.rs b/src/stream/src/executor/stateless_simple_agg.rs index cc7e876f1d2de..d0d00a6961371 100644 --- a/src/stream/src/executor/stateless_simple_agg.rs +++ b/src/stream/src/executor/stateless_simple_agg.rs @@ -22,7 +22,7 @@ use risingwave_expr::aggregate::{ build_retractable, AggCall, AggregateState, BoxedAggregateFunction, }; -use super::aggregation::{agg_call_filter_res, generate_agg_schema}; +use super::aggregation::agg_call_filter_res; use super::error::StreamExecutorError; use super::*; use crate::error::StreamResult; @@ -125,18 +125,10 @@ impl StatelessSimpleAggExecutor { pub fn new( ctx: ActorContextRef, input: Box, + info: ExecutorInfo, agg_calls: Vec, - pk_indices: PkIndices, - executor_id: u64, ) -> StreamResult { - let schema = generate_agg_schema(input.as_ref(), &agg_calls, None); - let info = ExecutorInfo { - schema, - pk_indices, - identity: format!("StatelessSimpleAggExecutor-{}", executor_id), - }; let aggs = agg_calls.iter().map(build_retractable).try_collect()?; - Ok(StatelessSimpleAggExecutor { _ctx: ctx, input, @@ -156,6 +148,7 @@ mod tests { use risingwave_common::catalog::schema_test_utils; use super::*; + use crate::executor::test_utils::agg_executor::generate_agg_schema; use crate::executor::test_utils::MockSource; use crate::executor::{Executor, StatelessSimpleAggExecutor}; @@ -168,14 +161,19 @@ mod tests { tx.push_barrier(3, false); let agg_calls = vec![AggCall::from_pretty("(count:int8)")]; + let schema = generate_agg_schema(&source, &agg_calls, None); + let info = ExecutorInfo { + schema, + pk_indices: vec![], + identity: "StatelessSimpleAggExecutor".to_string(), + }; let simple_agg = Box::new( StatelessSimpleAggExecutor::new( ActorContext::create(123), Box::new(source), + info, agg_calls, - vec![], - 1, ) .unwrap(), ); @@ -221,14 +219,19 @@ mod tests { AggCall::from_pretty("(sum:int8 $0:int8)"), AggCall::from_pretty("(sum:int8 $1:int8)"), ]; + let schema = generate_agg_schema(&source, &agg_calls, None); + let info = ExecutorInfo { + schema, + pk_indices: vec![], + identity: "StatelessSimpleAggExecutor".to_string(), + }; let simple_agg = Box::new( StatelessSimpleAggExecutor::new( ActorContext::create(123), Box::new(source), + info, agg_calls, - vec![], - 1, ) .unwrap(), ); diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index ec3b5f861e2d0..3b206c0eb1997 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -279,7 +279,7 @@ pub mod agg_executor { use std::sync::atomic::AtomicU64; use std::sync::Arc; - use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId}; + use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema, TableId}; use risingwave_common::hash::SerializedKey; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; @@ -295,10 +295,34 @@ pub mod agg_executor { use crate::executor::aggregation::AggStateStorage; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - ActorContext, ActorContextRef, BoxedExecutor, Executor, HashAggExecutor, PkIndices, - SimpleAggExecutor, + ActorContext, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, HashAggExecutor, + PkIndices, SimpleAggExecutor, }; + /// Generate agg executor's schema from `input`, `agg_calls` and `group_key_indices`. + /// For [`crate::executor::HashAggExecutor`], the group key indices should be provided. + pub fn generate_agg_schema( + input: &dyn Executor, + agg_calls: &[AggCall], + group_key_indices: Option<&[usize]>, + ) -> Schema { + let aggs = agg_calls + .iter() + .map(|agg| Field::unnamed(agg.return_type.clone())); + + let fields = if let Some(key_indices) = group_key_indices { + let keys = key_indices + .iter() + .map(|idx| input.schema().fields[*idx].clone()); + + keys.chain(aggs).collect() + } else { + aggs.collect() + }; + + Schema { fields } + } + /// Create state storage for the given agg call. /// Should infer the schema in the same way as `LogicalAgg::infer_stream_agg_state`. pub async fn create_agg_state_storage( @@ -448,13 +472,19 @@ pub mod agg_executor { ) .await; + let schema = generate_agg_schema(input.as_ref(), &agg_calls, Some(&group_key_indices)); + let info = ExecutorInfo { + schema, + pk_indices, + identity: format!("HashAggExecutor {:X}", executor_id), + }; + HashAggExecutor::::new(AggExecutorArgs { version: PbAggNodeVersion::Max, input, actor_ctx: ActorContext::create(123), - pk_indices, - executor_id, + info, extreme_cache_size, @@ -513,13 +543,19 @@ pub mod agg_executor { ) .await; + let schema = generate_agg_schema(input.as_ref(), &agg_calls, None); + let info = ExecutorInfo { + schema, + pk_indices, + identity: format!("SimpleAggExecutor {:X}", executor_id), + }; + SimpleAggExecutor::new(AggExecutorArgs { version: PbAggNodeVersion::Max, input, actor_ctx, - pk_indices, - executor_id, + info, extreme_cache_size: 1024, diff --git a/src/stream/src/from_proto/hash_agg.rs b/src/stream/src/from_proto/hash_agg.rs index 8160b1eb63658..153b6aea6a7d9 100644 --- a/src/stream/src/from_proto/hash_agg.rs +++ b/src/stream/src/from_proto/hash_agg.rs @@ -100,8 +100,11 @@ impl ExecutorBuilder for HashAggExecutorBuilder { input, actor_ctx: params.actor_context, - pk_indices: params.pk_indices, - executor_id: params.executor_id, + info: ExecutorInfo { + schema: params.schema, + pk_indices: params.pk_indices, + identity: params.identity, + }, extreme_cache_size: stream.config.developer.unsafe_extreme_cache_size, diff --git a/src/stream/src/from_proto/simple_agg.rs b/src/stream/src/from_proto/simple_agg.rs index acd618720c9f2..c0c5da2609042 100644 --- a/src/stream/src/from_proto/simple_agg.rs +++ b/src/stream/src/from_proto/simple_agg.rs @@ -61,8 +61,11 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder { input, actor_ctx: params.actor_context, - pk_indices: params.pk_indices, - executor_id: params.executor_id, + info: ExecutorInfo { + schema: params.schema, + pk_indices: params.pk_indices, + identity: params.identity, + }, extreme_cache_size: stream.config.developer.unsafe_extreme_cache_size, diff --git a/src/stream/src/from_proto/stateless_simple_agg.rs b/src/stream/src/from_proto/stateless_simple_agg.rs index 0654aea6da41f..e33098490c39e 100644 --- a/src/stream/src/from_proto/stateless_simple_agg.rs +++ b/src/stream/src/from_proto/stateless_simple_agg.rs @@ -39,9 +39,12 @@ impl ExecutorBuilder for StatelessSimpleAggExecutorBuilder { Ok(StatelessSimpleAggExecutor::new( params.actor_context, input, + ExecutorInfo { + schema: params.schema, + pk_indices: params.pk_indices, + identity: params.identity, + }, agg_calls, - params.pk_indices, - params.executor_id, )? .boxed()) } From 336dbe2e097843204684a6f2ed0882ecbfa5cf4d Mon Sep 17 00:00:00 2001 From: Huangjw <1223644280@qq.com> Date: Wed, 8 Nov 2023 10:13:25 +0800 Subject: [PATCH 02/77] fix(ci): fix specify risingwave image tag env (#13300) --- ci/scripts/integration-tests.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/scripts/integration-tests.sh b/ci/scripts/integration-tests.sh index 76146eb551d5f..f48ef99825ac3 100755 --- a/ci/scripts/integration-tests.sh +++ b/ci/scripts/integration-tests.sh @@ -41,7 +41,7 @@ cd integration_tests/scripts echo "--- case: ${case}, format: ${format}" -if [ -n "${RW_IMAGE_TAG}" ]; then +if [[ -n "${RW_IMAGE_TAG+x}" ]]; then export RW_IMAGE="ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_TAG}" echo Docker image: $RW_IMAGE fi From 7559073b7ce60b8b4046d5d026f97f5ea1960e3a Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Wed, 8 Nov 2023 10:50:11 +0800 Subject: [PATCH 03/77] feat: support `ALTER DATABASE/SCHEMA/TABLE/... xxx OWNER TO xxx` syntax (#13216) --- e2e_test/ddl/alter_owner.slt | 179 ++++++++++++++++++ proto/ddl_service.proto | 18 ++ src/frontend/src/catalog/catalog_service.rs | 8 + src/frontend/src/catalog/database_catalog.rs | 13 +- src/frontend/src/catalog/root_catalog.rs | 12 ++ src/frontend/src/catalog/schema_catalog.rs | 25 ++- .../system_catalog/rw_catalog/rw_databases.rs | 1 + .../system_catalog/rw_catalog/rw_schemas.rs | 1 + src/frontend/src/handler/alter_owner.rs | 93 +++++++++ src/frontend/src/handler/create_schema.rs | 2 +- src/frontend/src/handler/drop_database.rs | 4 +- src/frontend/src/handler/drop_schema.rs | 5 +- src/frontend/src/handler/mod.rs | 86 ++++++++- src/frontend/src/handler/privilege.rs | 25 ++- src/frontend/src/observer/observer_manager.rs | 2 + src/frontend/src/session.rs | 4 +- src/frontend/src/test_utils.rs | 21 ++ src/meta/service/src/ddl_service.rs | 15 ++ src/meta/src/manager/catalog/mod.rs | 92 +++++++++ src/meta/src/manager/catalog/user.rs | 6 +- src/meta/src/rpc/ddl_controller.rs | 15 +- src/rpc_client/src/meta_client.rs | 11 ++ src/sqlparser/src/ast/ddl.rs | 46 +++++ src/sqlparser/src/ast/mod.rs | 20 +- src/sqlparser/src/parser.rs | 74 +++++++- src/utils/pgwire/src/pg_response.rs | 2 + 26 files changed, 752 insertions(+), 28 deletions(-) create mode 100644 e2e_test/ddl/alter_owner.slt create mode 100644 src/frontend/src/handler/alter_owner.rs diff --git a/e2e_test/ddl/alter_owner.slt b/e2e_test/ddl/alter_owner.slt new file mode 100644 index 0000000000000..129443f9abaf2 --- /dev/null +++ b/e2e_test/ddl/alter_owner.slt @@ -0,0 +1,179 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +CREATE USER user1; + +statement ok +CREATE TABLE t (v1 INT primary key, v2 STRUCT>); + +statement ok +ALTER TABLE t OWNER TO user1; + +query TT +SELECT + pg_class.relname AS rel_name, + pg_roles.rolname AS owner +FROM + pg_class + JOIN pg_namespace ON pg_namespace.oid = pg_class.relnamespace + JOIN pg_roles ON pg_roles.oid = pg_class.relowner +WHERE + pg_namespace.nspname NOT LIKE 'pg_%' + AND pg_namespace.nspname != 'information_schema' + AND pg_class.relname = 't'; +---- +t user1 + +statement ok +CREATE VIEW v AS ( SELECT * FROM t WHERE v1 = 1); + +statement ok +ALTER VIEW v OWNER TO user1; + +query TT +SELECT + pg_class.relname AS rel_name, + pg_roles.rolname AS owner +FROM + pg_class + JOIN pg_namespace ON pg_namespace.oid = pg_class.relnamespace + JOIN pg_roles ON pg_roles.oid = pg_class.relowner +WHERE + pg_namespace.nspname NOT LIKE 'pg_%' + AND pg_namespace.nspname != 'information_schema' + AND pg_class.relname = 'v'; +---- +v user1 + +statement ok +CREATE MATERIALIZED VIEW mv AS SELECT v1, (t.v2).v1 AS v21 FROM t; + +statement ok +ALTER MATERIALIZED VIEW mv OWNER TO user1; + +query TT +SELECT + pg_class.relname AS rel_name, + pg_roles.rolname AS owner +FROM + pg_class + JOIN pg_namespace ON pg_namespace.oid = pg_class.relnamespace + JOIN pg_roles ON pg_roles.oid = pg_class.relowner +WHERE + pg_namespace.nspname NOT LIKE 'pg_%' + AND pg_namespace.nspname != 'information_schema' + AND pg_class.relname = 'mv'; +---- +mv user1 + +statement ok +CREATE SOURCE src (v INT) WITH ( + connector = 'datagen', + fields.v.kind = 'sequence', + fields.v.start = '1', + fields.v.end = '10', + datagen.rows.per.second='15', + datagen.split.num = '1' +) FORMAT PLAIN ENCODE JSON; + +statement ok +ALTER SOURCE src OWNER TO user1; + +query TT +SELECT + pg_class.relname AS rel_name, + pg_roles.rolname AS owner +FROM + pg_class + JOIN pg_namespace ON pg_namespace.oid = pg_class.relnamespace + JOIN pg_roles ON pg_roles.oid = pg_class.relowner +WHERE + pg_namespace.nspname NOT LIKE 'pg_%' + AND pg_namespace.nspname != 'information_schema' + AND pg_class.relname = 'src'; +---- +src user1 + +statement ok +CREATE SINK sink AS SELECT mv3.v1 AS v1, mv3.v21 AS v2 FROM mv AS mv3 WITH ( + connector = 'blackhole' +); + +statement ok +ALTER SINK sink OWNER TO user1; + +query TT +SELECT + pg_class.relname AS rel_name, + pg_roles.rolname AS owner +FROM + pg_class + JOIN pg_namespace ON pg_namespace.oid = pg_class.relnamespace + JOIN pg_roles ON pg_roles.oid = pg_class.relowner +WHERE + pg_namespace.nspname NOT LIKE 'pg_%' + AND pg_namespace.nspname != 'information_schema' + AND pg_class.relname = 'sink'; +---- +sink user1 + +statement ok +CREATE DATABASE d; + +statement ok +ALTER DATABASE d OWNER TO user1; + +query TT +SELECT + rw_databases.name AS db_name, + pg_roles.rolname AS owner +FROM + rw_databases + JOIN pg_roles ON pg_roles.oid = rw_databases.owner +WHERE + rw_databases.name = 'd'; +---- +d user1 + +statement ok +CREATE SCHEMA s; + +statement ok +ALTER SCHEMA s OWNER TO user1; + +query TT +SELECT + rw_schemas.name AS schema_name, + pg_roles.rolname AS owner +FROM + rw_schemas + JOIN pg_roles ON pg_roles.oid = rw_schemas.owner +WHERE + rw_schemas.name = 's'; +---- +s user1 + +statement ok +DROP SCHEMA s; + +statement ok +DROP DATABASE d; + +statement ok +DROP SINK sink; + +statement ok +DROP SOURCE src; + +statement ok +DROP MATERIALIZED VIEW mv; + +statement ok +DROP VIEW v; + +statement ok +DROP TABLE t; + +statement ok +DROP USER user1; diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 6d00225f32ec3..aaca4f332faae 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -182,6 +182,23 @@ message AlterRelationNameResponse { uint64 version = 2; } +message AlterOwnerRequest { + oneof object { + uint32 table_id = 1; + uint32 view_id = 2; + uint32 source_id = 3; + uint32 sink_id = 4; + uint32 schema_id = 5; + uint32 database_id = 6; + } + uint32 owner_id = 20; +} + +message AlterOwnerResponse { + common.Status status = 1; + uint64 version = 2; +} + message CreateFunctionRequest { catalog.Function function = 1; } @@ -355,6 +372,7 @@ service DdlService { rpc CreateTable(CreateTableRequest) returns (CreateTableResponse); rpc AlterRelationName(AlterRelationNameRequest) returns (AlterRelationNameResponse); rpc AlterSource(AlterSourceRequest) returns (AlterSourceResponse); + rpc AlterOwner(AlterOwnerRequest) returns (AlterOwnerResponse); rpc DropTable(DropTableRequest) returns (DropTableResponse); rpc RisectlListStateTables(RisectlListStateTablesRequest) returns (RisectlListStateTablesResponse); rpc CreateView(CreateViewRequest) returns (CreateViewResponse); diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index 657f02b260f9c..afb7ecad6c2d8 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -24,6 +24,7 @@ use risingwave_pb::catalog::{ PbComment, PbCreateType, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, }; +use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::ddl_service::alter_relation_name_request::Relation; use risingwave_pb::ddl_service::{create_connection_request, PbTableJobType}; use risingwave_pb::stream_plan::StreamFragmentGraph; @@ -155,6 +156,8 @@ pub trait CatalogWriter: Send + Sync { async fn alter_sink_name(&self, sink_id: u32, sink_name: &str) -> Result<()>; async fn alter_source_name(&self, source_id: u32, source_name: &str) -> Result<()>; + + async fn alter_owner(&self, object: Object, owner_id: u32) -> Result<()>; } #[derive(Clone)] @@ -413,6 +416,11 @@ impl CatalogWriter for CatalogWriterImpl { .await?; self.wait_version(version).await } + + async fn alter_owner(&self, object: Object, owner_id: u32) -> Result<()> { + let version = self.meta_client.alter_owner(object, owner_id).await?; + self.wait_version(version).await + } } impl CatalogWriterImpl { diff --git a/src/frontend/src/catalog/database_catalog.rs b/src/frontend/src/catalog/database_catalog.rs index 76cfc0cd359c1..17ae8725bc80b 100644 --- a/src/frontend/src/catalog/database_catalog.rs +++ b/src/frontend/src/catalog/database_catalog.rs @@ -18,8 +18,10 @@ use itertools::Itertools; use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; use risingwave_pb::catalog::{PbDatabase, PbSchema}; +use super::OwnedByUserCatalog; use crate::catalog::schema_catalog::SchemaCatalog; use crate::catalog::{DatabaseId, SchemaId, TableId}; +use crate::user::UserId; #[derive(Clone, Debug)] pub struct DatabaseCatalog { @@ -93,6 +95,12 @@ impl DatabaseCatalog { .find(|schema| schema.get_table_by_id(table_id).is_some()) } + pub fn update_self(&mut self, prost: &PbDatabase) { + self.id = prost.id; + self.name = prost.name.clone(); + self.owner = prost.owner; + } + pub fn is_empty(&self) -> bool { self.schema_by_name.len() == 1 && self.schema_by_name.contains_key(PG_CATALOG_SCHEMA_NAME) } @@ -104,11 +112,14 @@ impl DatabaseCatalog { pub fn name(&self) -> &str { &self.name } +} - pub fn owner(&self) -> u32 { +impl OwnedByUserCatalog for DatabaseCatalog { + fn owner(&self) -> UserId { self.owner } } + impl From<&PbDatabase> for DatabaseCatalog { fn from(db: &PbDatabase) -> Self { Self { diff --git a/src/frontend/src/catalog/root_catalog.rs b/src/frontend/src/catalog/root_catalog.rs index f8fa09efa43e3..fde63498709b6 100644 --- a/src/frontend/src/catalog/root_catalog.rs +++ b/src/frontend/src/catalog/root_catalog.rs @@ -262,6 +262,18 @@ impl Catalog { self.table_by_id.insert(proto.id.into(), table); } + pub fn update_database(&mut self, proto: &PbDatabase) { + self.get_database_mut(proto.id).unwrap().update_self(proto); + } + + pub fn update_schema(&mut self, proto: &PbSchema) { + self.get_database_mut(proto.database_id) + .unwrap() + .get_schema_mut(proto.id) + .unwrap() + .update_self(proto); + } + pub fn update_index(&mut self, proto: &PbIndex) { self.get_database_mut(proto.database_id) .unwrap() diff --git a/src/frontend/src/catalog/schema_catalog.rs b/src/frontend/src/catalog/schema_catalog.rs index e42cbc639bba6..ab69c95180e01 100644 --- a/src/frontend/src/catalog/schema_catalog.rs +++ b/src/frontend/src/catalog/schema_catalog.rs @@ -23,6 +23,7 @@ use risingwave_pb::catalog::{ PbConnection, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, }; +use super::OwnedByUserCatalog; use crate::catalog::connection_catalog::ConnectionCatalog; use crate::catalog::function_catalog::FunctionCatalog; use crate::catalog::index_catalog::IndexCatalog; @@ -31,6 +32,7 @@ use crate::catalog::system_catalog::SystemTableCatalog; use crate::catalog::table_catalog::TableCatalog; use crate::catalog::view_catalog::ViewCatalog; use crate::catalog::{ConnectionId, DatabaseId, SchemaId, SinkId, SourceId, ViewId}; +use crate::user::UserId; #[derive(Clone, Debug)] pub struct SchemaCatalog { @@ -91,6 +93,13 @@ impl SchemaCatalog { .unwrap(); } + pub fn update_self(&mut self, prost: &PbSchema) { + self.id = prost.id; + self.database_id = prost.database_id; + self.name = prost.name.clone(); + self.owner = prost.owner; + } + pub fn update_table(&mut self, prost: &PbTable) -> Arc { let name = prost.name.clone(); let id = prost.id.into(); @@ -440,6 +449,14 @@ impl SchemaCatalog { self.table_by_id.get(table_id) } + pub fn get_view_by_name(&self, view_name: &str) -> Option<&Arc> { + self.view_by_name.get(view_name) + } + + pub fn get_view_by_id(&self, view_id: &ViewId) -> Option<&Arc> { + self.view_by_id.get(view_id) + } + pub fn get_source_by_name(&self, source_name: &str) -> Option<&Arc> { self.source_by_name.get(source_name) } @@ -481,10 +498,6 @@ impl SchemaCatalog { .map(|table| table.name.clone()) } - pub fn get_view_by_name(&self, view_name: &str) -> Option<&Arc> { - self.view_by_name.get(view_name) - } - pub fn get_function_by_name_args( &self, name: &str, @@ -533,8 +546,10 @@ impl SchemaCatalog { pub fn name(&self) -> String { self.name.clone() } +} - pub fn owner(&self) -> u32 { +impl OwnedByUserCatalog for SchemaCatalog { + fn owner(&self) -> UserId { self.owner } } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs index 13fa09d13b2bd..17957ffdcc9be 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs @@ -20,6 +20,7 @@ use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::OwnedByUserCatalog; pub const RW_DATABASES: BuiltinTable = BuiltinTable { name: "rw_databases", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs index 2302317bb048b..ea83304f62a08 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs @@ -20,6 +20,7 @@ use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::OwnedByUserCatalog; pub const RW_SCHEMAS: BuiltinTable = BuiltinTable { name: "rw_schemas", diff --git a/src/frontend/src/handler/alter_owner.rs b/src/frontend/src/handler/alter_owner.rs new file mode 100644 index 0000000000000..d0fc70202528e --- /dev/null +++ b/src/frontend/src/handler/alter_owner.rs @@ -0,0 +1,93 @@ +// 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 pgwire::pg_response::StatementType; +use risingwave_common::error::Result; +use risingwave_pb::ddl_service::alter_owner_request::Object; +use risingwave_sqlparser::ast::{Ident, ObjectName}; + +use super::{HandlerArgs, RwPgResponse}; +use crate::catalog::root_catalog::SchemaPath; +use crate::catalog::CatalogError; +use crate::Binder; + +pub async fn handle_alter_owner( + handler_args: HandlerArgs, + obj_name: ObjectName, + new_owner_name: Ident, + stmt_type: StatementType, +) -> Result { + let session = handler_args.session; + let db_name = session.database(); + let (schema_name, real_obj_name) = + Binder::resolve_schema_qualified_name(db_name, obj_name.clone())?; + let search_path = session.config().get_search_path(); + let user_name = &session.auth_context().user_name; + let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name); + + let object = { + let catalog_reader = session.env().catalog_reader().read_guard(); + match stmt_type { + StatementType::ALTER_TABLE | StatementType::ALTER_MATERIALIZED_VIEW => { + let (table, schema_name) = + catalog_reader.get_table_by_name(db_name, schema_path, &real_obj_name)?; + session.check_privilege_for_drop_alter(schema_name, &**table)?; + Object::TableId(table.id.table_id) + } + StatementType::ALTER_VIEW => { + let (view, schema_name) = + catalog_reader.get_view_by_name(db_name, schema_path, &real_obj_name)?; + session.check_privilege_for_drop_alter(schema_name, &**view)?; + Object::ViewId(view.id) + } + StatementType::ALTER_SOURCE => { + let (source, schema_name) = + catalog_reader.get_source_by_name(db_name, schema_path, &real_obj_name)?; + session.check_privilege_for_drop_alter(schema_name, &**source)?; + Object::SourceId(source.id) + } + StatementType::ALTER_SINK => { + let (sink, schema_name) = + catalog_reader.get_sink_by_name(db_name, schema_path, &real_obj_name)?; + session.check_privilege_for_drop_alter(schema_name, &**sink)?; + Object::SinkId(sink.id.sink_id) + } + StatementType::ALTER_DATABASE => { + let database = catalog_reader.get_database_by_name(&obj_name.real_value())?; + session.check_privilege_for_drop_alter_db_schema(database)?; + Object::DatabaseId(database.id()) + } + StatementType::ALTER_SCHEMA => { + let schema = catalog_reader.get_schema_by_name(db_name, &obj_name.real_value())?; + session.check_privilege_for_drop_alter_db_schema(schema)?; + Object::SchemaId(schema.id()) + } + _ => unreachable!(), + } + }; + + let new_owner_name = Binder::resolve_user_name(vec![new_owner_name].into())?; + let owner_id = session + .env() + .user_info_reader() + .read_guard() + .get_user_by_name(&new_owner_name) + .map(|u| u.id) + .ok_or(CatalogError::NotFound("user", new_owner_name))?; + + let catalog_writer = session.catalog_writer()?; + catalog_writer.alter_owner(object, owner_id).await?; + + Ok(RwPgResponse::empty_result(stmt_type)) +} diff --git a/src/frontend/src/handler/create_schema.rs b/src/frontend/src/handler/create_schema.rs index 962d59178bf0c..ee75c6a5374d2 100644 --- a/src/frontend/src/handler/create_schema.rs +++ b/src/frontend/src/handler/create_schema.rs @@ -21,7 +21,7 @@ use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; use crate::binder::Binder; -use crate::catalog::CatalogError; +use crate::catalog::{CatalogError, OwnedByUserCatalog}; use crate::handler::privilege::ObjectCheckItem; use crate::handler::HandlerArgs; diff --git a/src/frontend/src/handler/drop_database.rs b/src/frontend/src/handler/drop_database.rs index 9037caa328079..7081c8b33f106 100644 --- a/src/frontend/src/handler/drop_database.rs +++ b/src/frontend/src/handler/drop_database.rs @@ -59,9 +59,7 @@ pub async fn handle_drop_database( } }; - if session.user_id() != database.owner() { - return Err(ErrorCode::PermissionDenied("Do not have the privilege".to_string()).into()); - } + session.check_privilege_for_drop_alter_db_schema(&database)?; let catalog_writer = session.catalog_writer()?; catalog_writer.drop_database(database.id()).await?; diff --git a/src/frontend/src/handler/drop_schema.rs b/src/frontend/src/handler/drop_schema.rs index fc79c45bd5e04..c0c71b97d2de8 100644 --- a/src/frontend/src/handler/drop_schema.rs +++ b/src/frontend/src/handler/drop_schema.rs @@ -14,7 +14,6 @@ use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::catalog::is_system_schema; -use risingwave_common::error::ErrorCode::PermissionDenied; use risingwave_common::error::{ErrorCode, Result}; use risingwave_sqlparser::ast::{DropMode, ObjectName}; @@ -90,9 +89,7 @@ pub async fn handle_drop_schema( } }; - if session.user_id() != schema.owner() { - return Err(PermissionDenied("Do not have the privilege".to_string()).into()); - } + session.check_privilege_for_drop_alter_db_schema(&schema)?; let catalog_writer = session.catalog_writer()?; catalog_writer.drop_schema(schema.id()).await?; diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index fadac730fdbb7..748269a0bd58a 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -18,7 +18,7 @@ use std::task::{Context, Poll}; use futures::stream::{self, BoxStream}; use futures::{Stream, StreamExt}; -use pgwire::pg_response::StatementType::{ABORT, BEGIN, COMMIT, ROLLBACK, START_TRANSACTION}; +use pgwire::pg_response::StatementType::{self, ABORT, BEGIN, COMMIT, ROLLBACK, START_TRANSACTION}; use pgwire::pg_response::{PgResponse, PgResponseBuilder, RowSetResult}; use pgwire::pg_server::BoxedError; use pgwire::types::{Format, Row}; @@ -33,6 +33,7 @@ use crate::scheduler::{DistributedQueryStream, LocalQueryStream}; use crate::session::SessionImpl; use crate::utils::WithOptions; +mod alter_owner; mod alter_relation_rename; mod alter_source_column; mod alter_system; @@ -457,6 +458,30 @@ pub async fn handle( ) .await } + Statement::AlterDatabase { + name, + operation: AlterDatabaseOperation::ChangeOwner { new_owner_name }, + } => { + alter_owner::handle_alter_owner( + handler_args, + name, + new_owner_name, + StatementType::ALTER_DATABASE, + ) + .await + } + Statement::AlterSchema { + name, + operation: AlterSchemaOperation::ChangeOwner { new_owner_name }, + } => { + alter_owner::handle_alter_owner( + handler_args, + name, + new_owner_name, + StatementType::ALTER_SCHEMA, + ) + .await + } Statement::AlterTable { name, operation: @@ -475,6 +500,18 @@ pub async fn handle( ) .await } + Statement::AlterTable { + name, + operation: AlterTableOperation::ChangeOwner { new_owner_name }, + } => { + alter_owner::handle_alter_owner( + handler_args, + name, + new_owner_name, + StatementType::ALTER_TABLE, + ) + .await + } Statement::AlterIndex { name, operation: AlterIndexOperation::RenameIndex { index_name }, @@ -496,10 +533,45 @@ pub async fn handle( alter_relation_rename::handle_rename_view(handler_args, name, view_name).await } } + Statement::AlterView { + materialized, + name, + operation: AlterViewOperation::ChangeOwner { new_owner_name }, + } => { + if materialized { + alter_owner::handle_alter_owner( + handler_args, + name, + new_owner_name, + StatementType::ALTER_MATERIALIZED_VIEW, + ) + .await + } else { + alter_owner::handle_alter_owner( + handler_args, + name, + new_owner_name, + StatementType::ALTER_VIEW, + ) + .await + } + } Statement::AlterSink { name, operation: AlterSinkOperation::RenameSink { sink_name }, } => alter_relation_rename::handle_rename_sink(handler_args, name, sink_name).await, + Statement::AlterSink { + name, + operation: AlterSinkOperation::ChangeOwner { new_owner_name }, + } => { + alter_owner::handle_alter_owner( + handler_args, + name, + new_owner_name, + StatementType::ALTER_SINK, + ) + .await + } Statement::AlterSource { name, operation: AlterSourceOperation::RenameSource { source_name }, @@ -508,6 +580,18 @@ pub async fn handle( name, operation: operation @ AlterSourceOperation::AddColumn { .. }, } => alter_source_column::handle_alter_source_column(handler_args, name, operation).await, + Statement::AlterSource { + name, + operation: AlterSourceOperation::ChangeOwner { new_owner_name }, + } => { + alter_owner::handle_alter_owner( + handler_args, + name, + new_owner_name, + StatementType::ALTER_SOURCE, + ) + .await + } Statement::AlterSystem { param, value } => { alter_system::handle_alter_system(handler_args, param, value).await } diff --git a/src/frontend/src/handler/privilege.rs b/src/frontend/src/handler/privilege.rs index 24227a46ff2a6..39984929a5dea 100644 --- a/src/frontend/src/handler/privilege.rs +++ b/src/frontend/src/handler/privilege.rs @@ -196,7 +196,7 @@ impl SessionImpl { && !self.is_super_user() { return Err(PermissionDenied( - "Only the relation owner, the schema owner, and superuser can drop a relation." + "Only the relation owner, the schema owner, and superuser can drop or alter a relation." .to_string(), ) .into()); @@ -204,6 +204,29 @@ impl SessionImpl { Ok(()) } + + /// Check whether the user of the current session has the privilege to drop or alter the + /// `db_schema`, which is either a database or schema. + /// > Only the owner of the database, or a superuser, can drop a database. + /// > + /// > Reference: + /// > + /// > A schema can only be dropped by its owner or a superuser. + /// > + /// > Reference: + pub fn check_privilege_for_drop_alter_db_schema( + &self, + db_schema: &impl OwnedByUserCatalog, + ) -> Result<()> { + if self.user_id() != db_schema.owner() && !self.is_super_user() { + return Err(PermissionDenied( + "Only the owner, and superuser can drop or alter a schema or database.".to_string(), + ) + .into()); + } + + Ok(()) + } } #[cfg(test)] diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index 6d3586ef84cbe..84fac6da59b42 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -215,11 +215,13 @@ impl FrontendObserverNode { Info::Database(database) => match resp.operation() { Operation::Add => catalog_guard.create_database(database), Operation::Delete => catalog_guard.drop_database(database.id), + Operation::Update => catalog_guard.update_database(database), _ => panic!("receive an unsupported notify {:?}", resp), }, Info::Schema(schema) => match resp.operation() { Operation::Add => catalog_guard.create_schema(schema), Operation::Delete => catalog_guard.drop_schema(schema.database_id, schema.id), + Operation::Update => catalog_guard.update_schema(schema), _ => panic!("receive an unsupported notify {:?}", resp), }, Info::RelationGroup(relation_group) => { diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index df4e8b77a4974..0170057024621 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -65,7 +65,9 @@ use crate::binder::{Binder, BoundStatement, ResolveQualifiedNameError}; use crate::catalog::catalog_service::{CatalogReader, CatalogWriter, CatalogWriterImpl}; use crate::catalog::connection_catalog::ConnectionCatalog; use crate::catalog::root_catalog::Catalog; -use crate::catalog::{check_schema_writable, CatalogError, DatabaseId, SchemaId}; +use crate::catalog::{ + check_schema_writable, CatalogError, DatabaseId, OwnedByUserCatalog, SchemaId, +}; use crate::handler::extended_handle::{ handle_bind, handle_execute, handle_parse, Portal, PrepareStatement, }; diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 6f88ea41a1bbd..c2c6840c8dd77 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -34,6 +34,7 @@ use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ PbComment, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, Table, }; +use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::ddl_service::{create_connection_request, DdlProgress, PbTableJobType}; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ @@ -496,6 +497,26 @@ impl CatalogWriter for MockCatalogWriter { Ok(()) } + async fn alter_owner(&self, object: Object, owner_id: u32) -> Result<()> { + for database in self.catalog.read().iter_databases() { + for schema in database.iter_schemas() { + match object { + Object::TableId(table_id) => { + if let Some(table) = schema.get_table_by_id(&TableId::from(table_id)) { + let mut pb_table = table.to_prost(schema.id(), database.id()); + pb_table.owner = owner_id; + self.catalog.write().update_table(&pb_table); + return Ok(()); + } + } + _ => unreachable!(), + } + } + } + + Err(ErrorCode::ItemNotFound(format!("object not found: {:?}", object)).into()) + } + async fn alter_view_name(&self, _view_id: u32, _view_name: &str) -> Result<()> { unreachable!() } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 4d996d7849924..c63f81da472c3 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -652,6 +652,21 @@ impl DdlService for DdlServiceImpl { })) } + async fn alter_owner( + &self, + request: Request, + ) -> Result, Status> { + let AlterOwnerRequest { object, owner_id } = request.into_inner(); + let version = self + .ddl_controller + .run_command(DdlCommand::AlterTableOwner(object.unwrap(), owner_id)) + .await?; + Ok(Response::new(AlterOwnerResponse { + status: None, + version, + })) + } + async fn get_ddl_progress( &self, _request: Request, diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 48cb07073d8e5..04bc743ae5348 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -37,6 +37,7 @@ use risingwave_pb::catalog::{ Comment, Connection, CreateType, Database, Function, Index, PbStreamJobStatus, Schema, Sink, Source, StreamJobStatus, Table, View, }; +use risingwave_pb::ddl_service::alter_owner_request; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, Object}; use risingwave_pb::user::update_user_request::UpdateField; @@ -1765,6 +1766,97 @@ impl CatalogManager { Ok(version) } + pub async fn alter_owner( + &self, + object: alter_owner_request::Object, + owner_id: UserId, + ) -> MetaResult { + let core = &mut *self.core.lock().await; + let database_core = &mut core.database; + let user_core = &mut core.user; + + let notify_info; + let old_owner_id; + match object { + alter_owner_request::Object::TableId(table_id) => { + database_core.ensure_table_id(table_id)?; + let mut tables = BTreeMapTransaction::new(&mut database_core.tables); + let mut table = tables.get_mut(table_id).unwrap(); + old_owner_id = table.owner; + table.owner = owner_id; + notify_info = Info::RelationGroup(RelationGroup { + relations: vec![Relation { + relation_info: Some(RelationInfo::Table(table.clone())), + }], + }); + commit_meta!(self, tables)?; + } + alter_owner_request::Object::ViewId(view_id) => { + database_core.ensure_view_id(view_id)?; + let mut views = BTreeMapTransaction::new(&mut database_core.views); + let mut view = views.get_mut(view_id).unwrap(); + old_owner_id = view.owner; + view.owner = owner_id; + notify_info = Info::RelationGroup(RelationGroup { + relations: vec![Relation { + relation_info: Some(RelationInfo::View(view.clone())), + }], + }); + commit_meta!(self, views)?; + } + alter_owner_request::Object::SourceId(source_id) => { + database_core.ensure_source_id(source_id)?; + let mut sources = BTreeMapTransaction::new(&mut database_core.sources); + let mut source = sources.get_mut(source_id).unwrap(); + old_owner_id = source.owner; + source.owner = owner_id; + notify_info = Info::RelationGroup(RelationGroup { + relations: vec![Relation { + relation_info: Some(RelationInfo::Source(source.clone())), + }], + }); + commit_meta!(self, sources)?; + } + alter_owner_request::Object::SinkId(sink_id) => { + database_core.ensure_sink_id(sink_id)?; + let mut sinks = BTreeMapTransaction::new(&mut database_core.sinks); + let mut sink = sinks.get_mut(sink_id).unwrap(); + old_owner_id = sink.owner; + sink.owner = owner_id; + notify_info = Info::RelationGroup(RelationGroup { + relations: vec![Relation { + relation_info: Some(RelationInfo::Sink(sink.clone())), + }], + }); + commit_meta!(self, sinks)?; + } + alter_owner_request::Object::DatabaseId(database_id) => { + database_core.ensure_database_id(database_id)?; + let mut databases = BTreeMapTransaction::new(&mut database_core.databases); + let mut database = databases.get_mut(database_id).unwrap(); + old_owner_id = database.owner; + database.owner = owner_id; + notify_info = Info::Database(database.clone()); + commit_meta!(self, databases)?; + } + alter_owner_request::Object::SchemaId(schema_id) => { + database_core.ensure_schema_id(schema_id)?; + let mut schemas = BTreeMapTransaction::new(&mut database_core.schemas); + let mut schema = schemas.get_mut(schema_id).unwrap(); + old_owner_id = schema.owner; + schema.owner = owner_id; + notify_info = Info::Schema(schema.clone()); + commit_meta!(self, schemas)?; + } + }; + user_core.increase_ref(owner_id); + user_core.decrease_ref(old_owner_id); + + let version = self.notify_frontend(Operation::Update, notify_info).await; + + Ok(version) + } + pub async fn alter_index_name( &self, index_id: IndexId, diff --git a/src/meta/src/manager/catalog/user.rs b/src/meta/src/manager/catalog/user.rs index 8037938937015..ff68ef21cbcb0 100644 --- a/src/meta/src/manager/catalog/user.rs +++ b/src/meta/src/manager/catalog/user.rs @@ -111,7 +111,11 @@ impl UserManager { pub fn decrease_ref_count(&mut self, user_id: UserId, count: usize) { match self.catalog_create_ref_count.entry(user_id) { Entry::Occupied(mut o) => { - assert!(*o.get_mut() >= count); + assert!( + *o.get() >= count, + "Attempted to decrease ref_count by {} but current ref_count is only {}. UserId: {:?}", + count, *o.get(), user_id + ); *o.get_mut() -= count; if *o.get() == 0 { o.remove_entry(); diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 9866061a14628..38ed3f16ae37f 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -27,6 +27,7 @@ use risingwave_pb::catalog::connection::private_link_service::PbPrivateLinkProvi use risingwave_pb::catalog::{ connection, Comment, Connection, CreateType, Database, Function, Schema, Source, Table, View, }; +use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::ddl_service::alter_relation_name_request::Relation; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::stream_plan::StreamFragmentGraph as StreamFragmentGraphProto; @@ -39,7 +40,7 @@ use crate::barrier::BarrierManagerRef; use crate::manager::{ CatalogManagerRef, ClusterManagerRef, ConnectionId, DatabaseId, FragmentManagerRef, FunctionId, IdCategory, IndexId, LocalNotification, MetaSrvEnv, NotificationVersion, RelationIdEnum, - SchemaId, SinkId, SourceId, StreamingClusterInfo, StreamingJob, TableId, ViewId, + SchemaId, SinkId, SourceId, StreamingClusterInfo, StreamingJob, TableId, UserId, ViewId, IGNORED_NOTIFICATION_VERSION, }; use crate::model::{StreamEnvironment, TableFragments}; @@ -102,6 +103,7 @@ pub enum DdlCommand { ReplaceTable(StreamingJob, StreamFragmentGraphProto, ColIndexMapping), AlterRelationName(Relation, String), AlterSourceColumn(Source), + AlterTableOwner(Object, UserId), CreateConnection(Connection), DropConnection(ConnectionId), CommentOn(Comment), @@ -255,6 +257,9 @@ impl DdlController { DdlCommand::AlterRelationName(relation, name) => { ctrl.alter_relation_name(relation, &name).await } + DdlCommand::AlterTableOwner(object, owner_id) => { + ctrl.alter_owner(object, owner_id).await + } DdlCommand::CreateConnection(connection) => { ctrl.create_connection(connection).await } @@ -1127,6 +1132,14 @@ impl DdlController { } } + async fn alter_owner( + &self, + object: Object, + owner_id: UserId, + ) -> MetaResult { + self.catalog_manager.alter_owner(object, owner_id).await + } + pub async fn wait(&self) -> MetaResult<()> { let timeout_secs = 30 * 60; for _ in 0..timeout_secs { diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 24b2d4c73e90a..0b9de6439d19b 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -47,6 +47,7 @@ use risingwave_pb::cloud_service::cloud_service_client::CloudServiceClient; use risingwave_pb::cloud_service::*; use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; use risingwave_pb::connector_service::sink_coordination_service_client::SinkCoordinationServiceClient; +use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::ddl_service::alter_relation_name_request::Relation; use risingwave_pb::ddl_service::ddl_service_client::DdlServiceClient; use risingwave_pb::ddl_service::drop_table_request::SourceId; @@ -456,6 +457,15 @@ impl MetaClient { Ok(resp.version) } + pub async fn alter_owner(&self, object: Object, owner_id: u32) -> Result { + let request = AlterOwnerRequest { + object: Some(object), + owner_id, + }; + let resp = self.inner.alter_owner(request).await?; + Ok(resp.version) + } + pub async fn replace_table( &self, source: Option, @@ -1727,6 +1737,7 @@ macro_rules! for_all_meta_rpc { ,{ stream_client, list_actor_states, ListActorStatesRequest, ListActorStatesResponse } ,{ ddl_client, create_table, CreateTableRequest, CreateTableResponse } ,{ ddl_client, alter_relation_name, AlterRelationNameRequest, AlterRelationNameResponse } + ,{ ddl_client, alter_owner, AlterOwnerRequest, AlterOwnerResponse } ,{ ddl_client, create_materialized_view, CreateMaterializedViewRequest, CreateMaterializedViewResponse } ,{ ddl_client, create_view, CreateViewRequest, CreateViewResponse } ,{ ddl_client, create_source, CreateSourceRequest, CreateSourceResponse } diff --git a/src/sqlparser/src/ast/ddl.rs b/src/sqlparser/src/ast/ddl.rs index ae497d777c4d0..502b3230c89e9 100644 --- a/src/sqlparser/src/ast/ddl.rs +++ b/src/sqlparser/src/ast/ddl.rs @@ -23,6 +23,20 @@ use serde::{Deserialize, Serialize}; use crate::ast::{display_comma_separated, display_separated, DataType, Expr, Ident, ObjectName}; use crate::tokenizer::Token; +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +#[cfg_attr(feature = "visitor", derive(Visit, VisitMut))] +pub enum AlterDatabaseOperation { + ChangeOwner { new_owner_name: Ident }, +} + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +#[cfg_attr(feature = "visitor", derive(Visit, VisitMut))] +pub enum AlterSchemaOperation { + ChangeOwner { new_owner_name: Ident }, +} + /// An `ALTER TABLE` (`Statement::AlterTable`) operation #[derive(Debug, Clone, PartialEq, Eq, Hash)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] @@ -89,6 +103,7 @@ pub enum AlterIndexOperation { #[cfg_attr(feature = "visitor", derive(Visit, VisitMut))] pub enum AlterViewOperation { RenameView { view_name: ObjectName }, + ChangeOwner { new_owner_name: Ident }, } #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -96,6 +111,7 @@ pub enum AlterViewOperation { #[cfg_attr(feature = "visitor", derive(Visit, VisitMut))] pub enum AlterSinkOperation { RenameSink { sink_name: ObjectName }, + ChangeOwner { new_owner_name: Ident }, } #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -104,6 +120,27 @@ pub enum AlterSinkOperation { pub enum AlterSourceOperation { RenameSource { source_name: ObjectName }, AddColumn { column_def: ColumnDef }, + ChangeOwner { new_owner_name: Ident }, +} + +impl fmt::Display for AlterDatabaseOperation { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + AlterDatabaseOperation::ChangeOwner { new_owner_name } => { + write!(f, "OWNER TO {}", new_owner_name) + } + } + } +} + +impl fmt::Display for AlterSchemaOperation { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + AlterSchemaOperation::ChangeOwner { new_owner_name } => { + write!(f, "OWNER TO {}", new_owner_name) + } + } + } } impl fmt::Display for AlterTableOperation { @@ -178,6 +215,9 @@ impl fmt::Display for AlterViewOperation { AlterViewOperation::RenameView { view_name } => { write!(f, "RENAME TO {view_name}") } + AlterViewOperation::ChangeOwner { new_owner_name } => { + write!(f, "OWNER TO {}", new_owner_name) + } } } } @@ -188,6 +228,9 @@ impl fmt::Display for AlterSinkOperation { AlterSinkOperation::RenameSink { sink_name } => { write!(f, "RENAME TO {sink_name}") } + AlterSinkOperation::ChangeOwner { new_owner_name } => { + write!(f, "OWNER TO {}", new_owner_name) + } } } } @@ -201,6 +244,9 @@ impl fmt::Display for AlterSourceOperation { AlterSourceOperation::AddColumn { column_def } => { write!(f, "ADD COLUMN {column_def}") } + AlterSourceOperation::ChangeOwner { new_owner_name } => { + write!(f, "OWNER TO {}", new_owner_name) + } } } } diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index fade5cfddda7e..d96016334b72c 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -32,8 +32,8 @@ use serde::{Deserialize, Serialize}; pub use self::data_type::{DataType, StructField}; pub use self::ddl::{ - AlterColumnOperation, AlterTableOperation, ColumnDef, ColumnOption, ColumnOptionDef, - ReferentialAction, SourceWatermark, TableConstraint, + AlterColumnOperation, AlterDatabaseOperation, AlterSchemaOperation, AlterTableOperation, + ColumnDef, ColumnOption, ColumnOptionDef, ReferentialAction, SourceWatermark, TableConstraint, }; pub use self::operator::{BinaryOperator, QualifiedOperator, UnaryOperator}; pub use self::query::{ @@ -1135,6 +1135,16 @@ pub enum Statement { append_only: bool, params: CreateFunctionBody, }, + /// ALTER DATABASE + AlterDatabase { + name: ObjectName, + operation: AlterDatabaseOperation, + }, + /// ALTER SCHEMA + AlterSchema { + name: ObjectName, + operation: AlterSchemaOperation, + }, /// ALTER TABLE AlterTable { /// Table name @@ -1601,6 +1611,12 @@ impl fmt::Display for Statement { ), Statement::CreateSink { stmt } => write!(f, "CREATE SINK {}", stmt,), Statement::CreateConnection { stmt } => write!(f, "CREATE CONNECTION {}", stmt,), + Statement::AlterDatabase { name, operation } => { + write!(f, "ALTER DATABASE {} {}", name, operation) + } + Statement::AlterSchema { name, operation } => { + write!(f, "ALTER SCHEMA {} {}", name, operation) + } Statement::AlterTable { name, operation } => { write!(f, "ALTER TABLE {} {}", name, operation) } diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 190c94a72a199..70050d920cdf3 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -26,7 +26,8 @@ use itertools::Itertools; use tracing::{debug, instrument}; use crate::ast::ddl::{ - AlterIndexOperation, AlterSinkOperation, AlterViewOperation, SourceWatermark, + AlterDatabaseOperation, AlterIndexOperation, AlterSchemaOperation, AlterSinkOperation, + AlterViewOperation, SourceWatermark, }; use crate::ast::{ParseTo, *}; use crate::keywords::{self, Keyword}; @@ -2783,7 +2784,11 @@ impl Parser { } pub fn parse_alter(&mut self) -> Result { - if self.parse_keyword(Keyword::TABLE) { + if self.parse_keyword(Keyword::DATABASE) { + self.parse_alter_database() + } else if self.parse_keyword(Keyword::SCHEMA) { + self.parse_alter_schema() + } else if self.parse_keyword(Keyword::TABLE) { self.parse_alter_table() } else if self.parse_keyword(Keyword::INDEX) { self.parse_alter_index() @@ -2801,12 +2806,46 @@ impl Parser { self.parse_alter_system() } else { self.expected( - "TABLE, INDEX, MATERIALIZED, VIEW, SINK, SOURCE, USER or SYSTEM after ALTER", + "DATABASE, SCHEMA, TABLE, INDEX, MATERIALIZED, VIEW, SINK, SOURCE, USER or SYSTEM after ALTER", self.peek_token(), ) } } + pub fn parse_alter_database(&mut self) -> Result { + let database_name = self.parse_object_name()?; + let operation = if self.parse_keywords(&[Keyword::OWNER, Keyword::TO]) { + let owner_name: Ident = self.parse_identifier()?; + AlterDatabaseOperation::ChangeOwner { + new_owner_name: owner_name, + } + } else { + return self.expected("OWNER TO after ALTER DATABASE", self.peek_token()); + }; + + Ok(Statement::AlterDatabase { + name: database_name, + operation, + }) + } + + pub fn parse_alter_schema(&mut self) -> Result { + let schema_name = self.parse_object_name()?; + let operation = if self.parse_keywords(&[Keyword::OWNER, Keyword::TO]) { + let owner_name: Ident = self.parse_identifier()?; + AlterSchemaOperation::ChangeOwner { + new_owner_name: owner_name, + } + } else { + return self.expected("OWNER TO after ALTER SCHEMA", self.peek_token()); + }; + + Ok(Statement::AlterSchema { + name: schema_name, + operation, + }) + } + pub fn parse_alter_user(&mut self) -> Result { Ok(Statement::AlterUser(AlterUserStatement::parse_to(self)?)) } @@ -2890,7 +2929,10 @@ impl Parser { }; AlterTableOperation::AlterColumn { column_name, op } } else { - return self.expected("ADD, RENAME or DROP after ALTER TABLE", self.peek_token()); + return self.expected( + "ADD, RENAME, OWNER TO or DROP after ALTER TABLE", + self.peek_token(), + ); }; Ok(Statement::AlterTable { name: table_name, @@ -2926,10 +2968,15 @@ impl Parser { } else { return self.expected("TO after RENAME", self.peek_token()); } + } else if self.parse_keywords(&[Keyword::OWNER, Keyword::TO]) { + let owner_name: Ident = self.parse_identifier()?; + AlterViewOperation::ChangeOwner { + new_owner_name: owner_name, + } } else { return self.expected( &format!( - "RENAME after ALTER {}VIEW", + "RENAME or OWNER TO after ALTER {}VIEW", if materialized { "MATERIALIZED " } else { "" } ), self.peek_token(), @@ -2952,8 +2999,13 @@ impl Parser { } else { return self.expected("TO after RENAME", self.peek_token()); } + } else if self.parse_keywords(&[Keyword::OWNER, Keyword::TO]) { + let owner_name: Ident = self.parse_identifier()?; + AlterSinkOperation::ChangeOwner { + new_owner_name: owner_name, + } } else { - return self.expected("RENAME after ALTER SINK", self.peek_token()); + return self.expected("RENAME or OWNER TO after ALTER SINK", self.peek_token()); }; Ok(Statement::AlterSink { @@ -2976,8 +3028,16 @@ impl Parser { let _if_not_exists = self.parse_keywords(&[Keyword::IF, Keyword::NOT, Keyword::EXISTS]); let column_def = self.parse_column_def()?; AlterSourceOperation::AddColumn { column_def } + } else if self.parse_keywords(&[Keyword::OWNER, Keyword::TO]) { + let owner_name: Ident = self.parse_identifier()?; + AlterSourceOperation::ChangeOwner { + new_owner_name: owner_name, + } } else { - return self.expected("RENAME | ADD COLUMN after ALTER SOURCE", self.peek_token()); + return self.expected( + "RENAME, ADD COLUMN or OWNER TO after ALTER SOURCE", + self.peek_token(), + ); }; Ok(Statement::AlterSource { diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 802f651ce4298..048919076db83 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -67,6 +67,8 @@ pub enum StatementType { DROP_DATABASE, DROP_USER, DROP_CONNECTION, + ALTER_DATABASE, + ALTER_SCHEMA, ALTER_INDEX, ALTER_VIEW, ALTER_TABLE, From c8f33ae615d75274cac604619f09c82428fc3fd0 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 7 Nov 2023 22:03:30 -0500 Subject: [PATCH 04/77] feat(expr): support list and struct in `jsonb_agg`/`jsonb_object_agg` (#13299) --- e2e_test/streaming/aggregate/jsonb_agg.slt | 18 ++++--- src/expr/impl/src/aggregate/jsonb_agg.rs | 57 +++++----------------- src/expr/macro/src/gen.rs | 18 ++++--- 3 files changed, 34 insertions(+), 59 deletions(-) diff --git a/e2e_test/streaming/aggregate/jsonb_agg.slt b/e2e_test/streaming/aggregate/jsonb_agg.slt index 18cb80cc69085..55930879970ec 100644 --- a/e2e_test/streaming/aggregate/jsonb_agg.slt +++ b/e2e_test/streaming/aggregate/jsonb_agg.slt @@ -2,7 +2,7 @@ statement ok SET RW_IMPLICIT_FLUSH TO true; statement ok -create table t(v1 boolean, v2 int, v3 varchar, v4 jsonb); +create table t(v1 boolean, v2 int, v3 varchar, v4 jsonb, v5 struct, v6 struct[]); statement ok create materialized view mv_tmp as @@ -16,28 +16,30 @@ create materialized view mv1 as select jsonb_agg(v1 order by v2) as j1, jsonb_agg(v2 order by v2) as j2, - jsonb_object_agg(v3, v4) as j3 + jsonb_object_agg(v3, v4) as j3, + jsonb_agg(v5 order by v2) as j4, + jsonb_object_agg(v3, v6) as j5 from t; statement ok insert into t values - (null, 2, 'bbb', null), - (false, 1, 'ccc', 'null'); + (null, 2, 'bbb', null, row(3, array[1, 2, 3]), null), + (false, 1, 'ccc', 'null', null, array[row(21, 'v21'), row(22, 'v22')]); query TTT select * from mv1; ---- -[false, null] [1, 2] {"bbb": null, "ccc": null} +[false, null] [1, 2] {"bbb": null, "ccc": null} [null, {"key": 3, "val": [1, 2, 3]}] {"bbb": null, "ccc": [{"key": 21, "val": "v21"}, {"key": 22, "val": "v22"}]} statement ok insert into t values - (true, 0, 'bbb', '999'), - (true, 8, 'ddd', '{"foo": "bar"}'); + (true, 0, 'bbb', '999', row(1, array[1, 3]), array[row(31, 'v31')]), + (true, 8, 'ddd', '{"foo": "bar"}', row(9, array[10]), null); query TTT select * from mv1; ---- -[true, false, null, true] [0, 1, 2, 8] {"bbb": 999, "ccc": null, "ddd": {"foo": "bar"}} +[true, false, null, true] [0, 1, 2, 8] {"bbb": 999, "ccc": null, "ddd": {"foo": "bar"}} [{"key": 1, "val": [1, 3]}, null, {"key": 3, "val": [1, 2, 3]}, {"key": 9, "val": [10]}] {"bbb": [{"key": 31, "val": "v31"}], "ccc": [{"key": 21, "val": "v21"}, {"key": 22, "val": "v22"}], "ddd": null} statement ok drop materialized view mv1; diff --git a/src/expr/impl/src/aggregate/jsonb_agg.rs b/src/expr/impl/src/aggregate/jsonb_agg.rs index dfc51c4d81581..cbe2cfbd44587 100644 --- a/src/expr/impl/src/aggregate/jsonb_agg.rs +++ b/src/expr/impl/src/aggregate/jsonb_agg.rs @@ -13,72 +13,39 @@ // limitations under the License. use risingwave_common::estimate_size::EstimateSize; -use risingwave_common::types::{DataType, JsonbVal, ScalarImpl}; +use risingwave_common::types::{JsonbVal, ScalarImpl}; use risingwave_expr::aggregate::AggStateDyn; +use risingwave_expr::expr::Context; use risingwave_expr::{aggregate, ExprError, Result}; use crate::scalar::ToJsonb; /// Collects all the input values, including nulls, into a JSON array. /// Values are converted to JSON as per `to_jsonb`. -#[aggregate("jsonb_agg(boolean) -> jsonb")] -#[aggregate("jsonb_agg(*int) -> jsonb")] -#[aggregate("jsonb_agg(*float) -> jsonb")] -#[aggregate("jsonb_agg(decimal) -> jsonb")] -#[aggregate("jsonb_agg(serial) -> jsonb")] -#[aggregate("jsonb_agg(int256) -> jsonb")] -#[aggregate("jsonb_agg(date) -> jsonb")] -#[aggregate("jsonb_agg(time) -> jsonb")] -#[aggregate("jsonb_agg(timestamp) -> jsonb")] -#[aggregate("jsonb_agg(timestamptz) -> jsonb")] -#[aggregate("jsonb_agg(interval) -> jsonb")] -#[aggregate("jsonb_agg(varchar) -> jsonb")] -#[aggregate("jsonb_agg(bytea) -> jsonb")] -#[aggregate("jsonb_agg(jsonb) -> jsonb")] -fn jsonb_agg(state: &mut JsonbArrayState, input: Option) -> Result<()> { - // FIXME(runji): - // None of the input types we currently support depend on `data_type` in `add_to`. - // So we just use a dummy type here. - // To get the correct type, we need to support `ctx: &Context` argument in `#[aggregate]`. - let data_type = &DataType::Int32; - - input.add_to(data_type, &mut state.0)?; +#[aggregate("jsonb_agg(*) -> jsonb")] +fn jsonb_agg( + state: &mut JsonbArrayState, + input: Option, + ctx: &Context, +) -> Result<()> { + input.add_to(&ctx.arg_types[0], &mut state.0)?; Ok(()) } /// Collects all the key/value pairs into a JSON object. -/// // TODO: support "any" type key /// // Key arguments are coerced to text; /// value arguments are converted as per `to_jsonb`. /// Values can be null, but keys cannot. -#[aggregate("jsonb_object_agg(varchar, boolean) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, *int) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, *float) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, decimal) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, serial) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, int256) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, date) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, time) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, timestamp) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, timestamptz) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, interval) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, varchar) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, bytea) -> jsonb")] -#[aggregate("jsonb_object_agg(varchar, jsonb) -> jsonb")] +#[aggregate("jsonb_object_agg(varchar, *) -> jsonb")] fn jsonb_object_agg( state: &mut JsonbObjectState, key: Option<&str>, value: Option, + ctx: &Context, ) -> Result<()> { - // FIXME(runji): - // None of the input types we currently support depend on `data_type` in `add_to`. - // So we just use a dummy type here. - // To get the correct type, we need to support `ctx: &Context` argument in `#[aggregate]`. - let data_type = &DataType::Int32; - let key = key.ok_or(ExprError::FieldNameNull)?; state.0.add_string(key); - value.add_to(data_type, &mut state.0)?; + value.add_to(&ctx.arg_types[1], &mut state.0)?; Ok(()) } diff --git a/src/expr/macro/src/gen.rs b/src/expr/macro/src/gen.rs index 5056d5dc24d11..325d9101cb2f2 100644 --- a/src/expr/macro/src/gen.rs +++ b/src/expr/macro/src/gen.rs @@ -653,14 +653,15 @@ impl FunctionAttr { }; let mut next_state = match user_fn { AggregateFnOrImpl::Fn(f) => { + let context = f.context.then(|| quote! { &self.context, }); let fn_name = format_ident!("{}", f.name); match f.retract { true => { - quote! { #fn_name(state, #args matches!(op, Op::Delete | Op::UpdateDelete)) } + quote! { #fn_name(state, #args matches!(op, Op::Delete | Op::UpdateDelete) #context) } } false => quote! {{ #panic_on_retract - #fn_name(state, #args) + #fn_name(state, #args #context) }}, } } @@ -781,20 +782,25 @@ impl FunctionAttr { use risingwave_common::buffer::Bitmap; use risingwave_common::estimate_size::EstimateSize; + use risingwave_expr::expr::Context; use risingwave_expr::Result; use risingwave_expr::aggregate::AggregateState; use risingwave_expr::codegen::async_trait; - #[derive(Clone)] + let context = Context { + return_type: agg.return_type.clone(), + arg_types: agg.args.arg_types().to_owned(), + }; + struct Agg { - return_type: DataType, + context: Context, #function_field } #[async_trait] impl risingwave_expr::aggregate::AggregateFunction for Agg { fn return_type(&self) -> DataType { - self.return_type.clone() + self.context.return_type.clone() } #create_state @@ -843,7 +849,7 @@ impl FunctionAttr { } Ok(Box::new(Agg { - return_type: agg.return_type.clone(), + context, #function_new })) } From fe8061811c184c6ddc48dc8eb0eb432933d6faa7 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Wed, 8 Nov 2023 11:42:39 +0800 Subject: [PATCH 05/77] chore(test): skip test with vendor because of invalid auth (#13305) Co-authored-by: Noel Kwan --- ci/scripts/pulsar-source-test.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/scripts/pulsar-source-test.sh b/ci/scripts/pulsar-source-test.sh index 255837513be05..9de314d30d425 100755 --- a/ci/scripts/pulsar-source-test.sh +++ b/ci/scripts/pulsar-source-test.sh @@ -47,7 +47,7 @@ cargo make ci-start ci-1cn-1fe echo "--- Run test" python3 -m pip install psycopg2-binary python3 e2e_test/source/pulsar/astra-streaming.py -python3 e2e_test/source/pulsar/streamnative-cloud.py +# python3 e2e_test/source/pulsar/streamnative-cloud.py echo "--- Kill cluster" cargo make ci-kill From f42a6121cfca88c7378688120b5c811ecfaa02e1 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Wed, 8 Nov 2023 14:19:17 +0800 Subject: [PATCH 06/77] feat(sink): support starrocks sink (#12681) --- integration_tests/starrocks-sink/README.md | 56 ++ .../append-only-sql/create_mv.sql | 7 + .../append-only-sql/create_sink.sql | 14 + .../append-only-sql/create_source.sql | 18 + .../starrocks-sink/docker-compose.yml | 78 +++ .../starrocks-sink/upsert/create_mv.sql | 7 + .../starrocks-sink/upsert/create_sink.sql | 14 + .../starrocks-sink/upsert/create_table.sql | 10 + .../upsert/insert_update_delete.sql | 8 + src/connector/src/common.rs | 1 - src/connector/src/sink/doris.rs | 252 ++++++-- src/connector/src/sink/doris_connector.rs | 423 ------------ .../src/sink/doris_starrocks_connector.rs | 320 +++++++++ src/connector/src/sink/encoder/json.rs | 7 +- src/connector/src/sink/mod.rs | 10 +- src/connector/src/sink/starrocks.rs | 610 ++++++++++++++++++ 16 files changed, 1369 insertions(+), 466 deletions(-) create mode 100644 integration_tests/starrocks-sink/README.md create mode 100644 integration_tests/starrocks-sink/append-only-sql/create_mv.sql create mode 100644 integration_tests/starrocks-sink/append-only-sql/create_sink.sql create mode 100644 integration_tests/starrocks-sink/append-only-sql/create_source.sql create mode 100644 integration_tests/starrocks-sink/docker-compose.yml create mode 100644 integration_tests/starrocks-sink/upsert/create_mv.sql create mode 100644 integration_tests/starrocks-sink/upsert/create_sink.sql create mode 100644 integration_tests/starrocks-sink/upsert/create_table.sql create mode 100644 integration_tests/starrocks-sink/upsert/insert_update_delete.sql delete mode 100644 src/connector/src/sink/doris_connector.rs create mode 100644 src/connector/src/sink/doris_starrocks_connector.rs create mode 100644 src/connector/src/sink/starrocks.rs diff --git a/integration_tests/starrocks-sink/README.md b/integration_tests/starrocks-sink/README.md new file mode 100644 index 0000000000000..f65b4b9406685 --- /dev/null +++ b/integration_tests/starrocks-sink/README.md @@ -0,0 +1,56 @@ +# Demo: Sinking to Starrocks + +In this demo, we want to showcase how RisingWave is able to sink data to Starrocks. + + +1. Launch the cluster: + +```sh +docker-compose up -d +``` + +The cluster contains a RisingWave cluster and its necessary dependencies, a datagen that generates the data, a Starrocks fe and be for sink. + +2. Create the Starrocks table via mysql: + +Login to mysql +```sh +docker compose exec starrocks-fe mysql -uroot -P9030 -h127.0.0.1 +``` + +Run the following queries to create database and table. +```sql +CREATE database demo; +use demo; + +CREATE table demo_bhv_table( + user_id int, + target_id text, + event_timestamp datetime +) ENGINE=OLAP +PRIMARY KEY(`user_id`) +DISTRIBUTED BY HASH(`user_id`) properties("replication_num" = "1"); + +CREATE USER 'users'@'%' IDENTIFIED BY '123456'; +GRANT ALL ON *.* TO 'users'@'%'; +``` + +3. Execute the SQL queries in sequence: + +- append-only sql: + - append-only/create_source.sql + - append-only/create_mv.sql + - append-only/create_sink.sql + +- upsert sql: + - upsert/create_table.sql + - upsert/create_mv.sql + - upsert/create_sink.sql + - upsert/insert_update_delete.sql + +We only support `upsert` with starrocks' `PRIMARY KEY` + +Run the following query +```sql +select user_id, count(*) from demo.demo_bhv_table group by user_id; +``` diff --git a/integration_tests/starrocks-sink/append-only-sql/create_mv.sql b/integration_tests/starrocks-sink/append-only-sql/create_mv.sql new file mode 100644 index 0000000000000..0a803f8a2762d --- /dev/null +++ b/integration_tests/starrocks-sink/append-only-sql/create_mv.sql @@ -0,0 +1,7 @@ +CREATE MATERIALIZED VIEW bhv_mv AS +SELECT + user_id, + target_id, + event_timestamp +FROM + user_behaviors; \ No newline at end of file diff --git a/integration_tests/starrocks-sink/append-only-sql/create_sink.sql b/integration_tests/starrocks-sink/append-only-sql/create_sink.sql new file mode 100644 index 0000000000000..56d1b227512de --- /dev/null +++ b/integration_tests/starrocks-sink/append-only-sql/create_sink.sql @@ -0,0 +1,14 @@ +CREATE SINK bhv_starrocks_sink +FROM + bhv_mv WITH ( + connector = 'starrocks', + type = 'append-only', + starrocks.host = 'starrocks-fe', + starrocks.mysqlport = '9030', + starrocks.httpport = '8030', + starrocks.user = 'users', + starrocks.password = '123456', + starrocks.database = 'demo', + starrocks.table = 'demo_bhv_table', + force_append_only='true' +); \ No newline at end of file diff --git a/integration_tests/starrocks-sink/append-only-sql/create_source.sql b/integration_tests/starrocks-sink/append-only-sql/create_source.sql new file mode 100644 index 0000000000000..c28c10f3616da --- /dev/null +++ b/integration_tests/starrocks-sink/append-only-sql/create_source.sql @@ -0,0 +1,18 @@ +CREATE table user_behaviors ( + user_id int, + target_id VARCHAR, + target_type VARCHAR, + event_timestamp TIMESTAMP, + behavior_type VARCHAR, + parent_target_type VARCHAR, + parent_target_id VARCHAR, + PRIMARY KEY(user_id) +) WITH ( + connector = 'datagen', + fields.user_id.kind = 'sequence', + fields.user_id.start = '1', + fields.user_id.end = '1000', + fields.user_name.kind = 'random', + fields.user_name.length = '10', + datagen.rows.per.second = '10' +) FORMAT PLAIN ENCODE JSON; \ No newline at end of file diff --git a/integration_tests/starrocks-sink/docker-compose.yml b/integration_tests/starrocks-sink/docker-compose.yml new file mode 100644 index 0000000000000..1933853c16915 --- /dev/null +++ b/integration_tests/starrocks-sink/docker-compose.yml @@ -0,0 +1,78 @@ +--- +version: "3" +services: + starrocks-fe: + image: starrocks/fe-ubuntu:latest + hostname: starrocks-fe + container_name: starrocks-fe + command: + /opt/starrocks/fe/bin/start_fe.sh + ports: + - 8030:8030 + - 9020:9020 + - 9030:9030 + healthcheck: + test: ["CMD", "curl", "-f", "http://localhost:9030"] + interval: 5s + timeout: 5s + retries: 30 + starrocks-be: + image: starrocks/be-ubuntu:latest + command: + - /bin/bash + - -c + - | + sleep 15s; mysql --connect-timeout 2 -h starrocks-fe -P9030 -uroot -e "alter system add backend \"starrocks-be:9050\";" + /opt/starrocks/be/bin/start_be.sh + ports: + - 8040:8040 + hostname: starrocks-be + container_name: starrocks-be + depends_on: + - starrocks-fe + compactor-0: + extends: + file: ../../docker/docker-compose.yml + service: compactor-0 + compute-node-0: + extends: + file: ../../docker/docker-compose.yml + service: compute-node-0 + etcd-0: + extends: + file: ../../docker/docker-compose.yml + service: etcd-0 + frontend-node-0: + extends: + file: ../../docker/docker-compose.yml + service: frontend-node-0 + grafana-0: + extends: + file: ../../docker/docker-compose.yml + service: grafana-0 + meta-node-0: + extends: + file: ../../docker/docker-compose.yml + service: meta-node-0 + minio-0: + extends: + file: ../../docker/docker-compose.yml + service: minio-0 + prometheus-0: + extends: + file: ../../docker/docker-compose.yml + service: prometheus-0 +volumes: + compute-node-0: + external: false + etcd-0: + external: false + grafana-0: + external: false + minio-0: + external: false + prometheus-0: + external: false + message_queue: + external: false +name: risingwave-compose \ No newline at end of file diff --git a/integration_tests/starrocks-sink/upsert/create_mv.sql b/integration_tests/starrocks-sink/upsert/create_mv.sql new file mode 100644 index 0000000000000..0a803f8a2762d --- /dev/null +++ b/integration_tests/starrocks-sink/upsert/create_mv.sql @@ -0,0 +1,7 @@ +CREATE MATERIALIZED VIEW bhv_mv AS +SELECT + user_id, + target_id, + event_timestamp +FROM + user_behaviors; \ No newline at end of file diff --git a/integration_tests/starrocks-sink/upsert/create_sink.sql b/integration_tests/starrocks-sink/upsert/create_sink.sql new file mode 100644 index 0000000000000..d7557bc1bd4fc --- /dev/null +++ b/integration_tests/starrocks-sink/upsert/create_sink.sql @@ -0,0 +1,14 @@ +CREATE SINK bhv_starrocks_sink +FROM + bhv_mv WITH ( + connector = 'starrocks', + type = 'upsert', + starrocks.host = 'starrocks-fe', + starrocks.mysqlport = '9030', + starrocks.httpport = '8030', + starrocks.user = 'users', + starrocks.password = '123456', + starrocks.database = 'demo', + starrocks.table = 'demo_bhv_table', + primary_key = 'user_id' +); \ No newline at end of file diff --git a/integration_tests/starrocks-sink/upsert/create_table.sql b/integration_tests/starrocks-sink/upsert/create_table.sql new file mode 100644 index 0000000000000..6c98f88a0b510 --- /dev/null +++ b/integration_tests/starrocks-sink/upsert/create_table.sql @@ -0,0 +1,10 @@ +CREATE table user_behaviors ( + user_id int, + target_id VARCHAR, + target_type VARCHAR, + event_timestamp TIMESTAMP, + behavior_type VARCHAR, + parent_target_type VARCHAR, + parent_target_id VARCHAR, + PRIMARY KEY(user_id) +); \ No newline at end of file diff --git a/integration_tests/starrocks-sink/upsert/insert_update_delete.sql b/integration_tests/starrocks-sink/upsert/insert_update_delete.sql new file mode 100644 index 0000000000000..73d5cda442258 --- /dev/null +++ b/integration_tests/starrocks-sink/upsert/insert_update_delete.sql @@ -0,0 +1,8 @@ +INSERT INTO user_behaviors VALUES(1,'1','1','2020-01-01 01:01:01','1','1','1'), +(2,'2','2','2020-01-01 01:01:02','2','2','2'), +(3,'3','3','2020-01-01 01:01:03','3','3','3'), +(4,'4','4','2020-01-01 01:01:04','4','4','4'); + +DELETE FROM user_behaviors WHERE user_id = 2; + +UPDATE user_behaviors SET target_id = 30 WHERE user_id = 3; \ No newline at end of file diff --git a/src/connector/src/common.rs b/src/connector/src/common.rs index cd8d99666c3e5..43f36527b54a4 100644 --- a/src/connector/src/common.rs +++ b/src/connector/src/common.rs @@ -404,7 +404,6 @@ impl KinesisCommon { Ok(KinesisClient::from_conf(builder.build())) } } - #[derive(Debug, Serialize, Deserialize)] pub struct UpsertMessage<'a> { #[serde(borrow)] diff --git a/src/connector/src/sink/doris.rs b/src/connector/src/sink/doris.rs index c8e2ddd23af31..f7f0faba368c4 100644 --- a/src/connector/src/sink/doris.rs +++ b/src/connector/src/sink/doris.rs @@ -17,6 +17,12 @@ use std::sync::Arc; use anyhow::anyhow; use async_trait::async_trait; +use base64::engine::general_purpose; +use base64::Engine; +use bytes::{BufMut, Bytes, BytesMut}; +use hyper::body::Body; +use hyper::{body, Client, Request}; +use hyper_tls::HttpsConnector; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; @@ -27,15 +33,14 @@ use serde_json::Value; use serde_with::serde_as; use with_options::WithOptions; -use super::doris_connector::{ - DorisField, DorisGet, DorisInsert, DorisInsertClient, DORIS_DELETE_SIGN, +use super::doris_starrocks_connector::{ + HeaderBuilder, InserterInner, InserterInnerBuilder, DORIS_DELETE_SIGN, DORIS_SUCCESS_STATUS, + POOL_IDLE_TIMEOUT, }; -use super::{SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use super::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; -use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriter, SinkWriterParam, -}; +use crate::sink::{DummySinkCommitCoordinator, Sink, SinkParam, SinkWriter, SinkWriterParam}; pub const DORIS_SINK: &str = "doris"; @@ -54,8 +59,8 @@ pub struct DorisCommon { } impl DorisCommon { - pub(crate) fn build_get_client(&self) -> DorisGet { - DorisGet::new( + pub(crate) fn build_get_client(&self) -> DorisSchemaClient { + DorisSchemaClient::new( self.url.clone(), self.table.clone(), self.database.clone(), @@ -224,9 +229,9 @@ pub struct DorisSinkWriter { pub config: DorisConfig, schema: Schema, pk_indices: Vec, - client: DorisInsertClient, + inseter_inner_builder: InserterInnerBuilder, is_append_only: bool, - insert: Option, + client: Option, row_encoder: JsonEncoder, } @@ -263,30 +268,31 @@ impl DorisSinkWriter { decimal_map.insert(s.name.clone(), v); } }); - let mut map = HashMap::new(); - map.insert("format".to_string(), "json".to_string()); - map.insert("read_json_by_line".to_string(), "true".to_string()); - let doris_insert_client = DorisInsertClient::new( + + let header_builder = HeaderBuilder::new() + .add_common_header() + .set_user_password(config.common.user.clone(), config.common.password.clone()) + .add_json_format() + .add_read_json_by_line(); + let header = if !is_append_only { + header_builder.add_hidden_column().build() + } else { + header_builder.build() + }; + + let doris_insert_builder = InserterInnerBuilder::new( config.common.url.clone(), config.common.database.clone(), config.common.table.clone(), - ) - .add_common_header() - .set_user_password(config.common.user.clone(), config.common.password.clone()) - .set_properties(map); - let mut doris_insert_client = if !is_append_only { - doris_insert_client.add_hidden_column() - } else { - doris_insert_client - }; - let insert = Some(doris_insert_client.build().await?); + header, + ); Ok(Self { config, schema: schema.clone(), pk_indices, - client: doris_insert_client, + inseter_inner_builder: doris_insert_builder, is_append_only, - insert, + client: None, row_encoder: JsonEncoder::new_with_doris( schema, None, @@ -302,7 +308,7 @@ impl DorisSinkWriter { continue; } let row_json_string = Value::Object(self.row_encoder.encode(row)?).to_string(); - self.insert + self.client .as_mut() .ok_or_else(|| SinkError::Doris("Can't find doris sink insert".to_string()))? .write(row_json_string.into()) @@ -322,7 +328,7 @@ impl DorisSinkWriter { ); let row_json_string = serde_json::to_string(&row_json_value) .map_err(|e| SinkError::Doris(format!("Json derialize error {:?}", e)))?; - self.insert + self.client .as_mut() .ok_or_else(|| { SinkError::Doris("Can't find doris sink insert".to_string()) @@ -338,7 +344,7 @@ impl DorisSinkWriter { ); let row_json_string = serde_json::to_string(&row_json_value) .map_err(|e| SinkError::Doris(format!("Json derialize error {:?}", e)))?; - self.insert + self.client .as_mut() .ok_or_else(|| { SinkError::Doris("Can't find doris sink insert".to_string()) @@ -355,7 +361,7 @@ impl DorisSinkWriter { ); let row_json_string = serde_json::to_string(&row_json_value) .map_err(|e| SinkError::Doris(format!("Json derialize error {:?}", e)))?; - self.insert + self.client .as_mut() .ok_or_else(|| { SinkError::Doris("Can't find doris sink insert".to_string()) @@ -372,8 +378,8 @@ impl DorisSinkWriter { #[async_trait] impl SinkWriter for DorisSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - if self.insert.is_none() { - self.insert = Some(self.client.build().await?); + if self.client.is_none() { + self.client = Some(DorisClient::new(self.inseter_inner_builder.build().await?)); } if self.is_append_only { self.append_only(chunk).await @@ -391,12 +397,12 @@ impl SinkWriter for DorisSinkWriter { } async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { - if self.insert.is_some() { - let insert = self - .insert + if self.client.is_some() { + let client = self + .client .take() .ok_or_else(|| SinkError::Doris("Can't find doris inserter".to_string()))?; - insert.finish().await?; + client.finish().await?; } Ok(()) } @@ -405,3 +411,177 @@ impl SinkWriter for DorisSinkWriter { Ok(()) } } + +pub struct DorisSchemaClient { + url: String, + table: String, + db: String, + user: String, + password: String, +} +impl DorisSchemaClient { + pub fn new(url: String, table: String, db: String, user: String, password: String) -> Self { + Self { + url, + table, + db, + user, + password, + } + } + + pub async fn get_schema_from_doris(&self) -> Result { + let uri = format!("{}/api/{}/{}/_schema", self.url, self.db, self.table); + let builder = Request::get(uri); + + let connector = HttpsConnector::new(); + let client = Client::builder() + .pool_idle_timeout(POOL_IDLE_TIMEOUT) + .build(connector); + + let request = builder + .header( + "Authorization", + format!( + "Basic {}", + general_purpose::STANDARD.encode(format!("{}:{}", self.user, self.password)) + ), + ) + .body(Body::empty()) + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + + let response = client + .request(request) + .await + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + + let raw_bytes = String::from_utf8(match body::to_bytes(response.into_body()).await { + Ok(bytes) => bytes.to_vec(), + Err(err) => return Err(SinkError::DorisStarrocksConnect(err.into())), + }) + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + + let json_map: HashMap = serde_json::from_str(&raw_bytes) + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + let json_data = if json_map.contains_key("code") && json_map.contains_key("msg") { + let data = json_map.get("data").ok_or_else(|| { + SinkError::DorisStarrocksConnect(anyhow::anyhow!("Can't find data")) + })?; + data.to_string() + } else { + raw_bytes + }; + let schema: DorisSchema = serde_json::from_str(&json_data).map_err(|err| { + SinkError::DorisStarrocksConnect(anyhow::anyhow!( + "Can't get schema from json {:?}", + err + )) + })?; + Ok(schema) + } +} +#[derive(Debug, Serialize, Deserialize)] +pub struct DorisSchema { + status: i32, + #[serde(rename = "keysType")] + pub keys_type: String, + pub properties: Vec, +} +#[derive(Debug, Serialize, Deserialize)] +pub struct DorisField { + pub name: String, + pub r#type: String, + comment: String, + pub precision: Option, + pub scale: Option, + aggregation_type: String, +} +impl DorisField { + pub fn get_decimal_pre_scale(&self) -> Option<(u8, u8)> { + if self.r#type.contains("DECIMAL") { + let a = self.precision.clone().unwrap().parse::().unwrap(); + let b = self.scale.clone().unwrap().parse::().unwrap(); + Some((a, b)) + } else { + None + } + } +} + +#[derive(Debug, Serialize, Deserialize)] +pub struct DorisInsertResultResponse { + #[serde(rename = "TxnId")] + txn_id: i64, + #[serde(rename = "Label")] + label: String, + #[serde(rename = "Status")] + status: String, + #[serde(rename = "TwoPhaseCommit")] + two_phase_commit: String, + #[serde(rename = "Message")] + message: String, + #[serde(rename = "NumberTotalRows")] + number_total_rows: i64, + #[serde(rename = "NumberLoadedRows")] + number_loaded_rows: i64, + #[serde(rename = "NumberFilteredRows")] + number_filtered_rows: i32, + #[serde(rename = "NumberUnselectedRows")] + number_unselected_rows: i32, + #[serde(rename = "LoadBytes")] + load_bytes: i64, + #[serde(rename = "LoadTimeMs")] + load_time_ms: i32, + #[serde(rename = "BeginTxnTimeMs")] + begin_txn_time_ms: i32, + #[serde(rename = "StreamLoadPutTimeMs")] + stream_load_put_time_ms: i32, + #[serde(rename = "ReadDataTimeMs")] + read_data_time_ms: i32, + #[serde(rename = "WriteDataTimeMs")] + write_data_time_ms: i32, + #[serde(rename = "CommitAndPublishTimeMs")] + commit_and_publish_time_ms: i32, + #[serde(rename = "ErrorURL")] + err_url: Option, +} + +pub struct DorisClient { + insert: InserterInner, + is_first_record: bool, +} +impl DorisClient { + pub fn new(insert: InserterInner) -> Self { + Self { + insert, + is_first_record: true, + } + } + + pub async fn write(&mut self, data: Bytes) -> Result<()> { + let mut data_build = BytesMut::new(); + if self.is_first_record { + self.is_first_record = false; + } else { + data_build.put_slice("\n".as_bytes()); + } + data_build.put_slice(&data); + self.insert.write(data_build.into()).await?; + Ok(()) + } + + pub async fn finish(self) -> Result { + let raw = self.insert.finish().await?; + let res: DorisInsertResultResponse = serde_json::from_slice(&raw) + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + + if !DORIS_SUCCESS_STATUS.contains(&res.status.as_str()) { + return Err(SinkError::DorisStarrocksConnect(anyhow::anyhow!( + "Insert error: {:?}, error url: {:?}", + res.message, + res.err_url + ))); + }; + Ok(res) + } +} diff --git a/src/connector/src/sink/doris_connector.rs b/src/connector/src/sink/doris_connector.rs deleted file mode 100644 index 116cd91d86542..0000000000000 --- a/src/connector/src/sink/doris_connector.rs +++ /dev/null @@ -1,423 +0,0 @@ -// 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 core::mem; -use core::time::Duration; -use std::collections::HashMap; - -use base64::engine::general_purpose; -use base64::Engine; -use bytes::{BufMut, Bytes, BytesMut}; -use http::request::Builder; -use hyper::body::{Body, Sender}; -use hyper::client::HttpConnector; -use hyper::{body, Client, Request, StatusCode}; -use hyper_tls::HttpsConnector; -use serde::{Deserialize, Serialize}; -use serde_json::Value; -use tokio::task::JoinHandle; - -use super::{Result, SinkError}; - -const BUFFER_SIZE: usize = 64 * 1024; -const MIN_CHUNK_SIZE: usize = BUFFER_SIZE - 1024; -const DORIS_SUCCESS_STATUS: [&str; 2] = ["Success", "Publish Timeout"]; -pub(crate) const DORIS_DELETE_SIGN: &str = "__DORIS_DELETE_SIGN__"; -const SEND_CHUNK_TIMEOUT: Duration = Duration::from_secs(10); -const WAIT_HANDDLE_TIMEOUT: Duration = Duration::from_secs(10); -const POOL_IDLE_TIMEOUT: Duration = Duration::from_secs(30); -pub struct DorisInsertClient { - url: String, - header: HashMap, - sender: Option, -} -impl DorisInsertClient { - pub fn new(url: String, db: String, table: String) -> Self { - let url = format!("{}/api/{}/{}/_stream_load", url, db, table); - Self { - url, - header: HashMap::default(), - sender: None, - } - } - - pub fn set_url(mut self, url: String) -> Self { - self.url = url; - self - } - - pub fn add_common_header(mut self) -> Self { - self.header - .insert("expect".to_string(), "100-continue".to_string()); - self - } - - /// The method is temporarily not in use, reserved for later use in 2PC. - /// Doris will generate a default, non-repeating label. - pub fn set_label(mut self, label: String) -> Self { - self.header.insert("label".to_string(), label); - self - } - - /// This method is only called during upsert operations. - pub fn add_hidden_column(mut self) -> Self { - self.header - .insert("hidden_columns".to_string(), DORIS_DELETE_SIGN.to_string()); - self - } - - /// The method is temporarily not in use, reserved for later use in 2PC. - pub fn enable_2_pc(mut self) -> Self { - self.header - .insert("two_phase_commit".to_string(), "true".to_string()); - self - } - - pub fn set_user_password(mut self, user: String, password: String) -> Self { - let auth = format!( - "Basic {}", - general_purpose::STANDARD.encode(format!("{}:{}", user, password)) - ); - self.header.insert("Authorization".to_string(), auth); - self - } - - /// The method is temporarily not in use, reserved for later use in 2PC. - pub fn set_txn_id(mut self, txn_id: i64) -> Self { - self.header - .insert("txn_operation".to_string(), txn_id.to_string()); - self - } - - /// The method is temporarily not in use, reserved for later use in 2PC. - pub fn add_commit(mut self) -> Self { - self.header - .insert("txn_operation".to_string(), "commit".to_string()); - self - } - - /// The method is temporarily not in use, reserved for later use in 2PC. - pub fn add_abort(mut self) -> Self { - self.header - .insert("txn_operation".to_string(), "abort".to_string()); - self - } - - /// This method is used to add custom message headers, such as the data import format. - pub fn set_properties(mut self, properties: HashMap) -> Self { - self.header.extend(properties); - self - } - - fn build_request_and_client( - &self, - uri: String, - ) -> (Builder, Client>) { - let mut builder = Request::put(uri); - for (k, v) in &self.header { - builder = builder.header(k, v); - } - - let connector = HttpsConnector::new(); - let client = Client::builder() - .pool_idle_timeout(POOL_IDLE_TIMEOUT) - .build(connector); - - (builder, client) - } - - pub async fn build(&mut self) -> Result { - let (builder, client) = self.build_request_and_client(self.url.clone()); - - let request_get_url = builder - .body(Body::empty()) - .map_err(|err| SinkError::Http(err.into()))?; - let resp = client - .request(request_get_url) - .await - .map_err(|err| SinkError::Http(err.into()))?; - let be_url = if resp.status() == StatusCode::TEMPORARY_REDIRECT { - resp.headers() - .get("location") - .ok_or_else(|| { - SinkError::Http(anyhow::anyhow!("Can't get doris BE url in header",)) - })? - .to_str() - .map_err(|err| { - SinkError::Http(anyhow::anyhow!( - "Can't get doris BE url in header {:?}", - err - )) - })? - } else { - return Err(SinkError::Http(anyhow::anyhow!("Can't get doris BE url",))); - }; - - let (builder, client) = self.build_request_and_client(be_url.to_string()); - let (sender, body) = Body::channel(); - let request = builder - .body(body) - .map_err(|err| SinkError::Http(err.into()))?; - let feature = client.request(request); - - let handle: JoinHandle> = tokio::spawn(async move { - let response = feature.await.map_err(|err| SinkError::Http(err.into()))?; - let status = response.status(); - let raw_string = String::from_utf8( - body::to_bytes(response.into_body()) - .await - .map_err(|err| SinkError::Http(err.into()))? - .to_vec(), - ) - .map_err(|err| SinkError::Http(err.into()))?; - - if status == StatusCode::OK && !raw_string.is_empty() { - let response: DorisInsertResultResponse = - serde_json::from_str(&raw_string).map_err(|err| SinkError::Http(err.into()))?; - Ok(response) - } else { - Err(SinkError::Http(anyhow::anyhow!( - "Failed connection {:?},{:?}", - status, - raw_string - ))) - } - }); - - Ok(DorisInsert::new(sender, handle)) - } -} - -pub struct DorisInsert { - sender: Option, - join_handle: Option>>, - buffer: BytesMut, - is_first_record: bool, -} -impl DorisInsert { - pub fn new(sender: Sender, join_handle: JoinHandle>) -> Self { - Self { - sender: Some(sender), - join_handle: Some(join_handle), - buffer: BytesMut::with_capacity(BUFFER_SIZE), - is_first_record: true, - } - } - - async fn send_chunk(&mut self) -> Result<()> { - if self.sender.is_none() { - return Ok(()); - } - - let chunk = mem::replace(&mut self.buffer, BytesMut::with_capacity(BUFFER_SIZE)); - - let is_timed_out = match tokio::time::timeout( - SEND_CHUNK_TIMEOUT, - self.sender.as_mut().unwrap().send_data(chunk.into()), - ) - .await - { - Ok(Ok(_)) => return Ok(()), - Ok(Err(_)) => false, - Err(_) => true, - }; - self.abort()?; - - let res = self.wait_handle().await; - - if is_timed_out { - Err(SinkError::Http(anyhow::anyhow!("timeout"))) - } else { - res?; - Err(SinkError::Http(anyhow::anyhow!("channel closed"))) - } - } - - fn abort(&mut self) -> Result<()> { - if let Some(sender) = self.sender.take() { - sender.abort(); - } - Ok(()) - } - - pub async fn write(&mut self, data: Bytes) -> Result<()> { - if self.is_first_record { - self.is_first_record = false; - } else { - self.buffer.put_slice("\n".as_bytes()); - } - self.buffer.put_slice(&data); - if self.buffer.len() >= MIN_CHUNK_SIZE { - self.send_chunk().await?; - } - Ok(()) - } - - async fn wait_handle(&mut self) -> Result { - let res = - match tokio::time::timeout(WAIT_HANDDLE_TIMEOUT, self.join_handle.as_mut().unwrap()) - .await - { - Ok(res) => res.map_err(|err| SinkError::Http(err.into()))??, - Err(err) => return Err(SinkError::Http(err.into())), - }; - if !DORIS_SUCCESS_STATUS.contains(&res.status.as_str()) { - return Err(SinkError::Http(anyhow::anyhow!( - "Insert error: {:?}, error url: {:?}", - res.message, - res.err_url - ))); - }; - Ok(res) - } - - pub async fn finish(mut self) -> Result { - if !self.buffer.is_empty() { - self.send_chunk().await?; - } - self.sender = None; - self.wait_handle().await - } -} - -pub struct DorisGet { - url: String, - table: String, - db: String, - user: String, - password: String, -} -impl DorisGet { - pub fn new(url: String, table: String, db: String, user: String, password: String) -> Self { - Self { - url, - table, - db, - user, - password, - } - } - - pub async fn get_schema_from_doris(&self) -> Result { - let uri = format!("{}/api/{}/{}/_schema", self.url, self.db, self.table); - let builder = Request::get(uri); - - let connector = HttpsConnector::new(); - let client = Client::builder() - .pool_idle_timeout(POOL_IDLE_TIMEOUT) - .build(connector); - - let request = builder - .header( - "Authorization", - format!( - "Basic {}", - general_purpose::STANDARD.encode(format!("{}:{}", self.user, self.password)) - ), - ) - .body(Body::empty()) - .map_err(|err| SinkError::Http(err.into()))?; - - let response = client - .request(request) - .await - .map_err(|err| SinkError::Http(err.into()))?; - - let raw_bytes = String::from_utf8(match body::to_bytes(response.into_body()).await { - Ok(bytes) => bytes.to_vec(), - Err(err) => return Err(SinkError::Http(err.into())), - }) - .map_err(|err| SinkError::Http(err.into()))?; - - let json_map: HashMap = - serde_json::from_str(&raw_bytes).map_err(|err| SinkError::Http(err.into()))?; - let json_data = if json_map.contains_key("code") && json_map.contains_key("msg") { - let data = json_map - .get("data") - .ok_or_else(|| SinkError::Http(anyhow::anyhow!("Can't find data")))?; - data.to_string() - } else { - raw_bytes - }; - let schema: DorisSchema = serde_json::from_str(&json_data).map_err(|err| { - SinkError::Http(anyhow::anyhow!("Can't get schema from json {:?}", err)) - })?; - Ok(schema) - } -} -#[derive(Debug, Serialize, Deserialize)] -pub struct DorisSchema { - status: i32, - #[serde(rename = "keysType")] - pub keys_type: String, - pub properties: Vec, -} -#[derive(Debug, Serialize, Deserialize)] -pub struct DorisField { - pub name: String, - pub r#type: String, - comment: String, - pub precision: Option, - pub scale: Option, - aggregation_type: String, -} -impl DorisField { - pub fn get_decimal_pre_scale(&self) -> Option<(u8, u8)> { - if self.r#type.contains("DECIMAL") { - let a = self.precision.clone().unwrap().parse::().unwrap(); - let b = self.scale.clone().unwrap().parse::().unwrap(); - Some((a, b)) - } else { - None - } - } -} - -#[derive(Debug, Serialize, Deserialize)] -pub struct DorisInsertResultResponse { - #[serde(rename = "TxnId")] - txn_id: i64, - #[serde(rename = "Label")] - label: String, - #[serde(rename = "Status")] - status: String, - #[serde(rename = "TwoPhaseCommit")] - two_phase_commit: String, - #[serde(rename = "Message")] - message: String, - #[serde(rename = "NumberTotalRows")] - number_total_rows: i64, - #[serde(rename = "NumberLoadedRows")] - number_loaded_rows: i64, - #[serde(rename = "NumberFilteredRows")] - number_filtered_rows: i32, - #[serde(rename = "NumberUnselectedRows")] - number_unselected_rows: i32, - #[serde(rename = "LoadBytes")] - load_bytes: i64, - #[serde(rename = "LoadTimeMs")] - load_time_ms: i32, - #[serde(rename = "BeginTxnTimeMs")] - begin_txn_time_ms: i32, - #[serde(rename = "StreamLoadPutTimeMs")] - stream_load_put_time_ms: i32, - #[serde(rename = "ReadDataTimeMs")] - read_data_time_ms: i32, - #[serde(rename = "WriteDataTimeMs")] - write_data_time_ms: i32, - #[serde(rename = "CommitAndPublishTimeMs")] - commit_and_publish_time_ms: i32, - #[serde(rename = "ErrorURL")] - err_url: Option, -} diff --git a/src/connector/src/sink/doris_starrocks_connector.rs b/src/connector/src/sink/doris_starrocks_connector.rs new file mode 100644 index 0000000000000..ca5dd8696d434 --- /dev/null +++ b/src/connector/src/sink/doris_starrocks_connector.rs @@ -0,0 +1,320 @@ +// 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 core::mem; +use core::time::Duration; +use std::collections::HashMap; + +use base64::engine::general_purpose; +use base64::Engine; +use bytes::{BufMut, Bytes, BytesMut}; +use http::request::Builder; +use hyper::body::{Body, Sender}; +use hyper::client::HttpConnector; +use hyper::{body, Client, Request, StatusCode}; +use hyper_tls::HttpsConnector; +use tokio::task::JoinHandle; + +use super::{Result, SinkError}; + +const BUFFER_SIZE: usize = 64 * 1024; +const MIN_CHUNK_SIZE: usize = BUFFER_SIZE - 1024; +pub(crate) const DORIS_SUCCESS_STATUS: [&str; 2] = ["Success", "Publish Timeout"]; +pub(crate) const DORIS_DELETE_SIGN: &str = "__DORIS_DELETE_SIGN__"; +pub(crate) const STARROCKS_DELETE_SIGN: &str = "__op"; + +const SEND_CHUNK_TIMEOUT: Duration = Duration::from_secs(10); +const WAIT_HANDDLE_TIMEOUT: Duration = Duration::from_secs(10); +pub(crate) const POOL_IDLE_TIMEOUT: Duration = Duration::from_secs(30); +const DORIS: &str = "doris"; +const STARROCKS: &str = "starrocks"; +pub struct HeaderBuilder { + header: HashMap, +} +impl Default for HeaderBuilder { + fn default() -> Self { + Self::new() + } +} +impl HeaderBuilder { + pub fn new() -> Self { + Self { + header: HashMap::default(), + } + } + + pub fn add_common_header(mut self) -> Self { + self.header + .insert("expect".to_string(), "100-continue".to_string()); + self + } + + /// The method is temporarily not in use, reserved for later use in 2PC. + /// Doris will generate a default, non-repeating label. + pub fn set_label(mut self, label: String) -> Self { + self.header.insert("label".to_string(), label); + self + } + + pub fn set_columns_name(mut self, columns_name: Vec<&str>) -> Self { + let columns_name_str = columns_name.join(","); + self.header.insert("columns".to_string(), columns_name_str); + self + } + + /// This method is only called during upsert operations. + pub fn add_hidden_column(mut self) -> Self { + self.header + .insert("hidden_columns".to_string(), DORIS_DELETE_SIGN.to_string()); + self + } + + /// The method is temporarily not in use, reserved for later use in 2PC. + /// Only use in Doris + pub fn enable_2_pc(mut self) -> Self { + self.header + .insert("two_phase_commit".to_string(), "true".to_string()); + self + } + + pub fn set_user_password(mut self, user: String, password: String) -> Self { + let auth = format!( + "Basic {}", + general_purpose::STANDARD.encode(format!("{}:{}", user, password)) + ); + self.header.insert("Authorization".to_string(), auth); + self + } + + /// The method is temporarily not in use, reserved for later use in 2PC. + /// Only use in Doris + pub fn set_txn_id(mut self, txn_id: i64) -> Self { + self.header + .insert("txn_operation".to_string(), txn_id.to_string()); + self + } + + /// The method is temporarily not in use, reserved for later use in 2PC. + /// Only use in Doris + pub fn add_commit(mut self) -> Self { + self.header + .insert("txn_operation".to_string(), "commit".to_string()); + self + } + + /// The method is temporarily not in use, reserved for later use in 2PC. + /// Only use in Doris + pub fn add_abort(mut self) -> Self { + self.header + .insert("txn_operation".to_string(), "abort".to_string()); + self + } + + pub fn add_json_format(mut self) -> Self { + self.header.insert("format".to_string(), "json".to_string()); + self + } + + /// Only use in Doris + pub fn add_read_json_by_line(mut self) -> Self { + self.header + .insert("read_json_by_line".to_string(), "true".to_string()); + self + } + + /// Only use in Starrocks + pub fn add_strip_outer_array(mut self) -> Self { + self.header + .insert("strip_outer_array".to_string(), "true".to_string()); + self + } + + pub fn build(self) -> HashMap { + self.header + } +} + +pub struct InserterInnerBuilder { + url: String, + header: HashMap, + sender: Option, +} +impl InserterInnerBuilder { + pub fn new(url: String, db: String, table: String, header: HashMap) -> Self { + let url = format!("{}/api/{}/{}/_stream_load", url, db, table); + + Self { + url, + sender: None, + header, + } + } + + fn build_request_and_client( + &self, + uri: String, + ) -> (Builder, Client>) { + let mut builder = Request::put(uri); + for (k, v) in &self.header { + builder = builder.header(k, v); + } + + let connector = HttpsConnector::new(); + let client = Client::builder() + .pool_idle_timeout(POOL_IDLE_TIMEOUT) + .build(connector); + + (builder, client) + } + + pub async fn build(&self) -> Result { + let (builder, client) = self.build_request_and_client(self.url.clone()); + let request_get_url = builder + .body(Body::empty()) + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + let resp = client + .request(request_get_url) + .await + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + let be_url = if resp.status() == StatusCode::TEMPORARY_REDIRECT { + resp.headers() + .get("location") + .ok_or_else(|| { + SinkError::DorisStarrocksConnect(anyhow::anyhow!( + "Can't get doris BE url in header", + )) + })? + .to_str() + .map_err(|err| { + SinkError::DorisStarrocksConnect(anyhow::anyhow!( + "Can't get doris BE url in header {:?}", + err + )) + })? + } else { + return Err(SinkError::DorisStarrocksConnect(anyhow::anyhow!( + "Can't get doris BE url", + ))); + }; + + let (builder, client) = self.build_request_and_client(be_url.to_string()); + let (sender, body) = Body::channel(); + let request = builder + .body(body) + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + let feature = client.request(request); + + let handle: JoinHandle>> = tokio::spawn(async move { + let response = feature + .await + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + let status = response.status(); + let raw = body::to_bytes(response.into_body()) + .await + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))? + .to_vec(); + if status == StatusCode::OK && !raw.is_empty() { + Ok(raw) + } else { + Err(SinkError::DorisStarrocksConnect(anyhow::anyhow!( + "Failed connection {:?},{:?}", + status, + String::from_utf8(raw) + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))? + ))) + } + }); + Ok(InserterInner::new(sender, handle)) + } +} + +pub struct InserterInner { + sender: Option, + join_handle: Option>>>, + buffer: BytesMut, +} +impl InserterInner { + pub fn new(sender: Sender, join_handle: JoinHandle>>) -> Self { + Self { + sender: Some(sender), + join_handle: Some(join_handle), + buffer: BytesMut::with_capacity(BUFFER_SIZE), + } + } + + async fn send_chunk(&mut self) -> Result<()> { + if self.sender.is_none() { + return Ok(()); + } + + let chunk = mem::replace(&mut self.buffer, BytesMut::with_capacity(BUFFER_SIZE)); + + let is_timed_out = match tokio::time::timeout( + SEND_CHUNK_TIMEOUT, + self.sender.as_mut().unwrap().send_data(chunk.into()), + ) + .await + { + Ok(Ok(_)) => return Ok(()), + Ok(Err(_)) => false, + Err(_) => true, + }; + self.abort()?; + + let res = self.wait_handle().await; + + if is_timed_out { + Err(SinkError::DorisStarrocksConnect(anyhow::anyhow!("timeout"))) + } else { + res?; + Err(SinkError::DorisStarrocksConnect(anyhow::anyhow!( + "channel closed" + ))) + } + } + + fn abort(&mut self) -> Result<()> { + if let Some(sender) = self.sender.take() { + sender.abort(); + } + Ok(()) + } + + pub async fn write(&mut self, data: Bytes) -> Result<()> { + self.buffer.put_slice(&data); + if self.buffer.len() >= MIN_CHUNK_SIZE { + self.send_chunk().await?; + } + Ok(()) + } + + async fn wait_handle(&mut self) -> Result> { + let res = + match tokio::time::timeout(WAIT_HANDDLE_TIMEOUT, self.join_handle.as_mut().unwrap()) + .await + { + Ok(res) => res.map_err(|err| SinkError::DorisStarrocksConnect(err.into()))??, + Err(err) => return Err(SinkError::DorisStarrocksConnect(err.into())), + }; + Ok(res) + } + + pub async fn finish(mut self) -> Result> { + if !self.buffer.is_empty() { + self.send_chunk().await?; + } + self.sender = None; + self.wait_handle().await + } +} diff --git a/src/connector/src/sink/encoder/json.rs b/src/connector/src/sink/encoder/json.rs index 7da859b8e9045..f85eb88acc233 100644 --- a/src/connector/src/sink/encoder/json.rs +++ b/src/connector/src/sink/encoder/json.rs @@ -196,15 +196,16 @@ fn datum_to_json_object( CustomJsonType::Doris(map) => { if !matches!(v, Decimal::Normalized(_)) { return Err(ArrayError::internal( - "doris can't support decimal Inf, -Inf, Nan".to_string(), + "doris/starrocks can't support decimal Inf, -Inf, Nan".to_string(), )); } let (p, s) = map.get(&field.name).unwrap(); v.rescale(*s as u32); let v_string = v.to_text(); - if v_string.len() > *p as usize { + let len = v_string.clone().replace(['.', '-'], "").len(); + if len > *p as usize { return Err(ArrayError::internal( - format!("rw Decimal's precision is large than doris max decimal len is {:?}, doris max is {:?}",v_string.len(),p))); + format!("rw Decimal's precision is large than doris/starrocks max decimal len is {:?}, doris max is {:?}",v_string.len(),p))); } json!(v_string) } diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 36fccc3798f1c..1136fb83085ba 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -19,7 +19,7 @@ pub mod catalog; pub mod clickhouse; pub mod coordinate; pub mod doris; -pub mod doris_connector; +pub mod doris_starrocks_connector; pub mod encoder; pub mod formatter; pub mod iceberg; @@ -30,6 +30,7 @@ pub mod nats; pub mod pulsar; pub mod redis; pub mod remote; +pub mod starrocks; pub mod test_sink; pub mod utils; pub mod writer; @@ -79,6 +80,7 @@ macro_rules! for_all_sinks { { ElasticSearch, $crate::sink::remote::ElasticSearchSink }, { Cassandra, $crate::sink::remote::CassandraSink }, { Doris, $crate::sink::doris::DorisSink }, + { Starrocks, $crate::sink::starrocks::StarrocksSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, { Test, $crate::sink::test_sink::TestSink } } @@ -416,14 +418,16 @@ pub enum SinkError { #[backtrace] anyhow::Error, ), - #[error("Doris http error: {0}")] - Http( + #[error("Doris/Starrocks connect error: {0}")] + DorisStarrocksConnect( #[source] #[backtrace] anyhow::Error, ), #[error("Doris error: {0}")] Doris(String), + #[error("Starrocks error: {0}")] + Starrocks(String), #[error("Pulsar error: {0}")] Pulsar( #[source] diff --git a/src/connector/src/sink/starrocks.rs b/src/connector/src/sink/starrocks.rs new file mode 100644 index 0000000000000..22597a80211eb --- /dev/null +++ b/src/connector/src/sink/starrocks.rs @@ -0,0 +1,610 @@ +// 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 std::collections::HashMap; +use std::sync::Arc; + +use anyhow::anyhow; +use async_trait::async_trait; +use bytes::Bytes; +use itertools::Itertools; +use mysql_async::prelude::Queryable; +use mysql_async::Opts; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::buffer::Bitmap; +use risingwave_common::catalog::Schema; +use risingwave_common::types::DataType; +use serde::Deserialize; +use serde_derive::Serialize; +use serde_json::Value; +use serde_with::serde_as; + +use super::doris_starrocks_connector::{ + HeaderBuilder, InserterInner, InserterInnerBuilder, DORIS_SUCCESS_STATUS, STARROCKS_DELETE_SIGN, +}; +use super::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use super::writer::LogSinkerOf; +use super::{SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use crate::sink::writer::SinkWriterExt; +use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; + +pub const STARROCKS_SINK: &str = "starrocks"; +const STARROCK_MYSQL_PREFER_SOCKET: &str = "false"; +const STARROCK_MYSQL_MAX_ALLOWED_PACKET: usize = 1024; +const STARROCK_MYSQL_WAIT_TIMEOUT: usize = 28800; + +#[derive(Deserialize, Serialize, Debug, Clone)] +pub struct StarrocksCommon { + #[serde(rename = "starrocks.host")] + pub host: String, + #[serde(rename = "starrocks.mysqlport")] + pub mysql_port: String, + #[serde(rename = "starrocks.httpport")] + pub http_port: String, + #[serde(rename = "starrocks.user")] + pub user: String, + #[serde(rename = "starrocks.password")] + pub password: String, + #[serde(rename = "starrocks.database")] + pub database: String, + #[serde(rename = "starrocks.table")] + pub table: String, +} + +#[serde_as] +#[derive(Clone, Debug, Deserialize)] +pub struct StarrocksConfig { + #[serde(flatten)] + pub common: StarrocksCommon, + + pub r#type: String, // accept "append-only" or "upsert" +} +impl StarrocksConfig { + pub fn from_hashmap(properties: HashMap) -> Result { + let config = + serde_json::from_value::(serde_json::to_value(properties).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) + } +} + +#[derive(Debug)] +pub struct StarrocksSink { + pub config: StarrocksConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, +} + +impl StarrocksSink { + pub fn new( + config: StarrocksConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + Ok(Self { + config, + schema, + pk_indices, + is_append_only, + }) + } +} + +impl StarrocksSink { + fn check_column_name_and_type( + &self, + starrocks_columns_desc: HashMap, + ) -> Result<()> { + let rw_fields_name = self.schema.fields(); + if rw_fields_name.len().ne(&starrocks_columns_desc.len()) { + return Err(SinkError::Starrocks("The length of the RisingWave column must be equal to the length of the starrocks column".to_string())); + } + + for i in rw_fields_name { + let value = starrocks_columns_desc.get(&i.name).ok_or_else(|| { + SinkError::Starrocks(format!( + "Column name don't find in starrocks, risingwave is {:?} ", + i.name + )) + })?; + if !Self::check_and_correct_column_type(&i.data_type, value.to_string())? { + return Err(SinkError::Starrocks(format!( + "Column type don't match, column name is {:?}. starrocks type is {:?} risingwave type is {:?} ",i.name,value,i.data_type + ))); + } + } + Ok(()) + } + + fn check_and_correct_column_type( + rw_data_type: &DataType, + starrocks_data_type: String, + ) -> Result { + match rw_data_type { + risingwave_common::types::DataType::Boolean => { + Ok(starrocks_data_type.contains("tinyint")) + } + risingwave_common::types::DataType::Int16 => { + Ok(starrocks_data_type.contains("smallint")) + } + risingwave_common::types::DataType::Int32 => Ok(starrocks_data_type.contains("int")), + risingwave_common::types::DataType::Int64 => Ok(starrocks_data_type.contains("bigint")), + risingwave_common::types::DataType::Float32 => { + Ok(starrocks_data_type.contains("float")) + } + risingwave_common::types::DataType::Float64 => { + Ok(starrocks_data_type.contains("double")) + } + risingwave_common::types::DataType::Decimal => { + Ok(starrocks_data_type.contains("decimal")) + } + risingwave_common::types::DataType::Date => Ok(starrocks_data_type.contains("date")), + risingwave_common::types::DataType::Varchar => { + Ok(starrocks_data_type.contains("varchar")) + } + risingwave_common::types::DataType::Time => Err(SinkError::Starrocks( + "starrocks can not support Time".to_string(), + )), + risingwave_common::types::DataType::Timestamp => { + Ok(starrocks_data_type.contains("datetime")) + } + risingwave_common::types::DataType::Timestamptz => Err(SinkError::Starrocks( + "starrocks can not support Timestamptz".to_string(), + )), + risingwave_common::types::DataType::Interval => Err(SinkError::Starrocks( + "starrocks can not support Interval".to_string(), + )), + // todo! Validate the type struct and list + risingwave_common::types::DataType::Struct(_) => Err(SinkError::Starrocks( + "starrocks can not support import struct".to_string(), + )), + risingwave_common::types::DataType::List(_) => { + Ok(starrocks_data_type.contains("unknown")) + } + risingwave_common::types::DataType::Bytea => Err(SinkError::Starrocks( + "starrocks can not support Bytea".to_string(), + )), + risingwave_common::types::DataType::Jsonb => Err(SinkError::Starrocks( + "starrocks can not support import json".to_string(), + )), + risingwave_common::types::DataType::Serial => { + Ok(starrocks_data_type.contains("bigint")) + } + risingwave_common::types::DataType::Int256 => Err(SinkError::Starrocks( + "starrocks can not support Int256".to_string(), + )), + } + } +} + +impl Sink for StarrocksSink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = LogSinkerOf; + + const SINK_NAME: &'static str = STARROCKS_SINK; + + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok(StarrocksSinkWriter::new( + self.config.clone(), + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + ) + .await? + .into_log_sinker(writer_param.sink_metrics)) + } + + async fn validate(&self) -> Result<()> { + if !self.is_append_only && self.pk_indices.is_empty() { + return Err(SinkError::Config(anyhow!( + "Primary key not defined for upsert starrocks sink (please define in `primary_key` field)"))); + } + // check reachability + let mut client = StarrocksSchemaClient::new( + self.config.common.host.clone(), + self.config.common.mysql_port.clone(), + self.config.common.table.clone(), + self.config.common.database.clone(), + self.config.common.user.clone(), + self.config.common.password.clone(), + ) + .await?; + let (read_model, pks) = client.get_pk_from_starrocks().await?; + + if !self.is_append_only && read_model.ne("PRIMARY_KEYS") { + return Err(SinkError::Config(anyhow!( + "If you want to use upsert, please set the keysType of starrocks to PRIMARY_KEY" + ))); + } + + for (index, filed) in self.schema.fields().iter().enumerate() { + if self.pk_indices.contains(&index) && !pks.contains(&filed.name) { + return Err(SinkError::Starrocks(format!( + "Can't find pk {:?} in starrocks", + filed.name + ))); + } + } + + let starrocks_columns_desc = client.get_columns_from_starrocks().await?; + + self.check_column_name_and_type(starrocks_columns_desc)?; + Ok(()) + } +} + +pub struct StarrocksSinkWriter { + pub config: StarrocksConfig, + schema: Schema, + pk_indices: Vec, + inserter_innet_builder: InserterInnerBuilder, + is_append_only: bool, + client: Option, + row_encoder: JsonEncoder, +} + +impl TryFrom for StarrocksSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = StarrocksConfig::from_hashmap(param.properties)?; + StarrocksSink::new( + config, + schema, + param.downstream_pk, + param.sink_type.is_append_only(), + ) + } +} + +impl StarrocksSinkWriter { + pub async fn new( + config: StarrocksConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + let mut decimal_map = HashMap::default(); + let starrocks_columns = StarrocksSchemaClient::new( + config.common.host.clone(), + config.common.mysql_port.clone(), + config.common.table.clone(), + config.common.database.clone(), + config.common.user.clone(), + config.common.password.clone(), + ) + .await? + .get_columns_from_starrocks() + .await?; + + for (name, column_type) in &starrocks_columns { + if column_type.contains("decimal") { + let decimal_all = column_type + .split("decimal(") + .last() + .ok_or_else(|| SinkError::Starrocks("must have last".to_string()))? + .split(')') + .next() + .ok_or_else(|| SinkError::Starrocks("must have next".to_string()))? + .split(',') + .collect_vec(); + let length = decimal_all + .first() + .ok_or_else(|| SinkError::Starrocks("must have next".to_string()))? + .parse::() + .map_err(|e| SinkError::Starrocks(format!("starrocks sink error {}", e)))?; + + let scale = decimal_all + .last() + .ok_or_else(|| SinkError::Starrocks("must have next".to_string()))? + .parse::() + .map_err(|e| SinkError::Starrocks(format!("starrocks sink error {}", e)))?; + decimal_map.insert(name.to_string(), (length, scale)); + } + } + + let builder = HeaderBuilder::new() + .add_common_header() + .set_user_password(config.common.user.clone(), config.common.password.clone()) + .add_json_format(); + let header = if !is_append_only { + let mut fields_name = schema.names_str(); + fields_name.push(STARROCKS_DELETE_SIGN); + builder.set_columns_name(fields_name).build() + } else { + builder.build() + }; + + let starrocks_insert_builder = InserterInnerBuilder::new( + format!("http://{}:{}", config.common.host, config.common.http_port), + config.common.database.clone(), + config.common.table.clone(), + header, + ); + Ok(Self { + config, + schema: schema.clone(), + pk_indices, + inserter_innet_builder: starrocks_insert_builder, + is_append_only, + client: None, + row_encoder: JsonEncoder::new_with_doris( + schema, + None, + TimestampHandlingMode::String, + decimal_map, + ), + }) + } + + async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { + for (op, row) in chunk.rows() { + if op != Op::Insert { + continue; + } + let row_json_string = Value::Object(self.row_encoder.encode(row)?).to_string(); + self.client + .as_mut() + .ok_or_else(|| { + SinkError::Starrocks("Can't find starrocks sink insert".to_string()) + })? + .write(row_json_string.into()) + .await?; + } + Ok(()) + } + + async fn upsert(&mut self, chunk: StreamChunk) -> Result<()> { + for (op, row) in chunk.rows() { + match op { + Op::Insert => { + let mut row_json_value = self.row_encoder.encode(row)?; + row_json_value.insert( + STARROCKS_DELETE_SIGN.to_string(), + Value::String("0".to_string()), + ); + let row_json_string = serde_json::to_string(&row_json_value).map_err(|e| { + SinkError::Starrocks(format!("Json derialize error {:?}", e)) + })?; + self.client + .as_mut() + .ok_or_else(|| { + SinkError::Starrocks("Can't find starrocks sink insert".to_string()) + })? + .write(row_json_string.into()) + .await?; + } + Op::Delete => { + let mut row_json_value = self.row_encoder.encode(row)?; + row_json_value.insert( + STARROCKS_DELETE_SIGN.to_string(), + Value::String("1".to_string()), + ); + let row_json_string = serde_json::to_string(&row_json_value).map_err(|e| { + SinkError::Starrocks(format!("Json derialize error {:?}", e)) + })?; + self.client + .as_mut() + .ok_or_else(|| { + SinkError::Starrocks("Can't find starrocks sink insert".to_string()) + })? + .write(row_json_string.into()) + .await?; + } + Op::UpdateDelete => {} + Op::UpdateInsert => { + let mut row_json_value = self.row_encoder.encode(row)?; + row_json_value.insert( + STARROCKS_DELETE_SIGN.to_string(), + Value::String("0".to_string()), + ); + let row_json_string = serde_json::to_string(&row_json_value).map_err(|e| { + SinkError::Starrocks(format!("Json derialize error {:?}", e)) + })?; + self.client + .as_mut() + .ok_or_else(|| { + SinkError::Starrocks("Can't find starrocks sink insert".to_string()) + })? + .write(row_json_string.into()) + .await?; + } + } + } + Ok(()) + } +} + +#[async_trait] +impl SinkWriter for StarrocksSinkWriter { + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + if self.client.is_none() { + self.client = Some(StarrocksClient::new( + self.inserter_innet_builder.build().await?, + )); + } + if self.is_append_only { + self.append_only(chunk).await + } else { + self.upsert(chunk).await + } + } + + async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { + Ok(()) + } + + async fn abort(&mut self) -> Result<()> { + Ok(()) + } + + async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { + if self.client.is_some() { + let client = self + .client + .take() + .ok_or_else(|| SinkError::Starrocks("Can't find starrocks inserter".to_string()))?; + client.finish().await?; + } + Ok(()) + } + + async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { + Ok(()) + } +} + +pub struct StarrocksSchemaClient { + table: String, + db: String, + conn: mysql_async::Conn, +} + +impl StarrocksSchemaClient { + pub async fn new( + host: String, + port: String, + table: String, + db: String, + user: String, + password: String, + ) -> Result { + let conn_uri = format!( + "mysql://{}:{}@{}:{}/{}?prefer_socket={}&max_allowed_packet={}&wait_timeout={}", + user, + password, + host, + port, + db, + STARROCK_MYSQL_PREFER_SOCKET, + STARROCK_MYSQL_MAX_ALLOWED_PACKET, + STARROCK_MYSQL_WAIT_TIMEOUT + ); + let pool = mysql_async::Pool::new( + Opts::from_url(&conn_uri) + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?, + ); + let conn = pool + .get_conn() + .await + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + + Ok(Self { table, db, conn }) + } + + pub async fn get_columns_from_starrocks(&mut self) -> Result> { + let query = format!("select column_name, column_type from information_schema.columns where table_name = {:?} and table_schema = {:?};",self.table,self.db); + let mut query_map: HashMap = HashMap::default(); + self.conn + .query_map(query, |(column_name, column_type)| { + query_map.insert(column_name, column_type) + }) + .await + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + Ok(query_map) + } + + pub async fn get_pk_from_starrocks(&mut self) -> Result<(String, String)> { + let query = format!("select table_model, primary_key from information_schema.tables_config where table_name = {:?} and table_schema = {:?};",self.table,self.db); + let table_mode_pk: (String, String) = self + .conn + .query_map(query, |(table_model, primary_key)| { + (table_model, primary_key) + }) + .await + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))? + .get(0) + .ok_or_else(|| { + SinkError::Starrocks(format!( + "Can't find schema with table {:?} and database {:?}", + self.table, self.db + )) + })? + .clone(); + Ok(table_mode_pk) + } +} + +#[derive(Debug, Serialize, Deserialize)] +pub struct StarrocksInsertResultResponse { + #[serde(rename = "TxnId")] + txn_id: i64, + #[serde(rename = "Label")] + label: String, + #[serde(rename = "Status")] + status: String, + #[serde(rename = "Message")] + message: String, + #[serde(rename = "NumberTotalRows")] + number_total_rows: i64, + #[serde(rename = "NumberLoadedRows")] + number_loaded_rows: i64, + #[serde(rename = "NumberFilteredRows")] + number_filtered_rows: i32, + #[serde(rename = "NumberUnselectedRows")] + number_unselected_rows: i32, + #[serde(rename = "LoadBytes")] + load_bytes: i64, + #[serde(rename = "LoadTimeMs")] + load_time_ms: i32, + #[serde(rename = "BeginTxnTimeMs")] + begin_txn_time_ms: i32, + #[serde(rename = "ReadDataTimeMs")] + read_data_time_ms: i32, + #[serde(rename = "WriteDataTimeMs")] + write_data_time_ms: i32, + #[serde(rename = "CommitAndPublishTimeMs")] + commit_and_publish_time_ms: i32, + #[serde(rename = "StreamLoadPlanTimeMs")] + stream_load_plan_time_ms: Option, +} + +pub struct StarrocksClient { + insert: InserterInner, +} +impl StarrocksClient { + pub fn new(insert: InserterInner) -> Self { + Self { insert } + } + + pub async fn write(&mut self, data: Bytes) -> Result<()> { + self.insert.write(data).await?; + Ok(()) + } + + pub async fn finish(self) -> Result { + let raw = self.insert.finish().await?; + let res: StarrocksInsertResultResponse = serde_json::from_slice(&raw) + .map_err(|err| SinkError::DorisStarrocksConnect(err.into()))?; + + if !DORIS_SUCCESS_STATUS.contains(&res.status.as_str()) { + return Err(SinkError::DorisStarrocksConnect(anyhow::anyhow!( + "Insert error: {:?}", + res.message, + ))); + }; + Ok(res) + } +} From 566bacf77bc43720b63911fcf47b3e87d873626a Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 8 Nov 2023 15:03:57 +0800 Subject: [PATCH 07/77] feat(sink): support jdbc sink async truncate (#13190) --- .../connector/api/sink/SinkWriter.java | 22 +- .../connector/api/sink/SinkWriterV1.java | 3 +- .../connector/SinkWriterStreamObserver.java | 22 +- .../connector/sink/jdbc/JDBCSinkTest.java | 18 +- .../AppendOnlyIcebergSinkWriter.java | 3 +- .../connector/UpsertIcebergSinkWriter.java | 3 +- .../com/risingwave/connector/JDBCSink.java | 29 +- .../risingwave/connector/JDBCSinkFactory.java | 3 +- proto/connector_service.proto | 6 + src/connector/src/sink/remote.rs | 263 +++++++++++++----- 10 files changed, 270 insertions(+), 102 deletions(-) diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkWriter.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkWriter.java index 385514c64ecec..a211d8f9750f0 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkWriter.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkWriter.java @@ -20,11 +20,31 @@ import java.util.Optional; public interface SinkWriter { + /** + * Begin writing an epoch. + * + * @param epoch + */ void beginEpoch(long epoch); - void write(Iterable rows); + /** + * Write a series of rows to the external sink. + * + * @return Flag to indicate whether the rows are written and persisting in the external sink. + * `true` means persisted. + */ + boolean write(Iterable rows); + /** + * Mark the end of the previous begun epoch. + * + * @param isCheckpoint `isCheckpoint` = `true` means that the RW kernel will do a checkpoint for + * data before this barrier. External sink should have its data persisted before it returns. + * @return Optionally return the metadata of this checkpoint. Only return some metadata for + * coordinated remote sink when `isCheckpoint` == `true`. + */ Optional barrier(boolean isCheckpoint); + /** Clean up */ void drop(); } diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkWriterV1.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkWriterV1.java index acdba21c5f94b..9ab7e71dc8bf6 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkWriterV1.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkWriterV1.java @@ -43,11 +43,12 @@ public SinkWriterV1 getInner() { public void beginEpoch(long epoch) {} @Override - public void write(Iterable rows) { + public boolean write(Iterable rows) { if (!hasBegun) { hasBegun = true; } this.inner.write(rows.iterator()); + return false; } @Override diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkWriterStreamObserver.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkWriterStreamObserver.java index d74e7c1848a18..1fb42d477f0fc 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkWriterStreamObserver.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkWriterStreamObserver.java @@ -125,13 +125,29 @@ public void onNext(ConnectorServiceProto.SinkWriterStreamRequest sinkTask) { .asRuntimeException(); } + boolean batchWritten; + try (CloseableIterable rowIter = deserializer.deserialize(batch)) { - sink.write( - new MonitoredRowIterable( - rowIter, connectorName, String.valueOf(sinkId))); + batchWritten = + sink.write( + new MonitoredRowIterable( + rowIter, connectorName, String.valueOf(sinkId))); } currentBatchId = batch.getBatchId(); + + if (batchWritten) { + responseObserver.onNext( + ConnectorServiceProto.SinkWriterStreamResponse.newBuilder() + .setBatch( + ConnectorServiceProto.SinkWriterStreamResponse + .BatchWrittenResponse.newBuilder() + .setEpoch(currentEpoch) + .setBatchId(currentBatchId) + .build()) + .build()); + } + LOG.debug("Batch {} written to epoch {}", currentBatchId, batch.getEpoch()); } else if (sinkTask.hasBarrier()) { if (!isInitialized()) { diff --git a/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/jdbc/JDBCSinkTest.java b/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/jdbc/JDBCSinkTest.java index da9b9d866583b..918ea46f9e49f 100644 --- a/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/jdbc/JDBCSinkTest.java +++ b/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/jdbc/JDBCSinkTest.java @@ -16,7 +16,6 @@ import static org.junit.Assert.*; -import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.risingwave.connector.JDBCSink; import com.risingwave.connector.JDBCSinkConfig; @@ -26,6 +25,7 @@ import com.risingwave.proto.Data.DataType.TypeName; import com.risingwave.proto.Data.Op; import java.sql.*; +import java.util.Arrays; import org.junit.Test; import org.testcontainers.containers.JdbcDatabaseContainer; import org.testcontainers.containers.MySQLContainer; @@ -84,7 +84,7 @@ static void testJDBCSync(JdbcDatabaseContainer container, TestType testType) Connection conn = sink.getConn(); sink.write( - Iterators.forArray( + Arrays.asList( new ArraySinkRow( Op.INSERT, 1, @@ -94,7 +94,7 @@ static void testJDBCSync(JdbcDatabaseContainer container, TestType testType) new Timestamp(1000000000), "{\"key\": \"password\", \"value\": \"Singularity123\"}", "I want to sleep".getBytes()))); - sink.sync(); + sink.barrier(true); Statement stmt = conn.createStatement(); try (var rs = stmt.executeQuery(String.format("SELECT * FROM %s", tableName))) { @@ -106,7 +106,7 @@ static void testJDBCSync(JdbcDatabaseContainer container, TestType testType) } sink.write( - Iterators.forArray( + Arrays.asList( new ArraySinkRow( Op.INSERT, 2, @@ -116,7 +116,7 @@ static void testJDBCSync(JdbcDatabaseContainer container, TestType testType) new Timestamp(1000000000), "{\"key\": \"password\", \"value\": \"Singularity123\"}", "I want to sleep".getBytes()))); - sink.sync(); + sink.barrier(true); try (var rs = stmt.executeQuery(String.format("SELECT * FROM %s", tableName))) { int count; for (count = 0; rs.next(); ) { @@ -126,7 +126,7 @@ static void testJDBCSync(JdbcDatabaseContainer container, TestType testType) } stmt.close(); - sink.sync(); + sink.barrier(true); sink.drop(); } @@ -144,7 +144,7 @@ static void testJDBCWrite(JdbcDatabaseContainer container, TestType testType) Statement stmt = conn.createStatement(); sink.write( - Iterators.forArray( + Arrays.asList( new ArraySinkRow( Op.INSERT, 1, @@ -171,7 +171,7 @@ static void testJDBCWrite(JdbcDatabaseContainer container, TestType testType) } sink.write( - Iterators.forArray( + Arrays.asList( new ArraySinkRow( Op.UPDATE_DELETE, 1, @@ -216,7 +216,7 @@ static void testJDBCWrite(JdbcDatabaseContainer container, TestType testType) assertFalse(rs.next()); } - sink.sync(); + sink.barrier(true); stmt.close(); } diff --git a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/AppendOnlyIcebergSinkWriter.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/AppendOnlyIcebergSinkWriter.java index 902fcd46dd244..7fa0a27f38536 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/AppendOnlyIcebergSinkWriter.java +++ b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/AppendOnlyIcebergSinkWriter.java @@ -52,7 +52,7 @@ public AppendOnlyIcebergSinkWriter( } @Override - public void write(Iterable rows) { + public boolean write(Iterable rows) { for (SinkRow row : rows) { switch (row.getOp()) { case INSERT: @@ -106,6 +106,7 @@ public void write(Iterable rows) { .asRuntimeException(); } } + return false; } @Override diff --git a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSinkWriter.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSinkWriter.java index 55a71a3089091..9c2b7c7745341 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSinkWriter.java +++ b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSinkWriter.java @@ -139,7 +139,7 @@ private List> getKeyFromRow(SinkRow row) { } @Override - public void write(Iterable rows) { + public boolean write(Iterable rows) { for (SinkRow row : rows) { if (row.size() != tableSchema.getColumnNames().length) { throw Status.FAILED_PRECONDITION @@ -188,6 +188,7 @@ public void write(Iterable rows) { .asRuntimeException(); } } + return false; } @Override diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java index ea8429536c03c..220caca4d13fe 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java @@ -16,8 +16,9 @@ import com.risingwave.connector.api.TableSchema; import com.risingwave.connector.api.sink.SinkRow; -import com.risingwave.connector.api.sink.SinkWriterBase; +import com.risingwave.connector.api.sink.SinkWriter; import com.risingwave.connector.jdbc.JdbcDialect; +import com.risingwave.proto.ConnectorServiceProto; import com.risingwave.proto.Data; import io.grpc.Status; import java.sql.*; @@ -25,7 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class JDBCSink extends SinkWriterBase { +public class JDBCSink implements SinkWriter { private static final String ERROR_REPORT_TEMPLATE = "Error when exec %s, message %s"; private final JdbcDialect jdbcDialect; @@ -33,6 +34,8 @@ public class JDBCSink extends SinkWriterBase { private final Connection conn; private final List pkColumnNames; + private final TableSchema tableSchema; + public static final String JDBC_COLUMN_NAME_KEY = "COLUMN_NAME"; public static final String JDBC_DATA_TYPE_KEY = "DATA_TYPE"; @@ -45,7 +48,7 @@ public class JDBCSink extends SinkWriterBase { private static final Logger LOG = LoggerFactory.getLogger(JDBCSink.class); public JDBCSink(JDBCSinkConfig config, TableSchema tableSchema) { - super(tableSchema); + this.tableSchema = tableSchema; var jdbcUrl = config.getJdbcUrl().toLowerCase(); var factory = JdbcUtils.getDialectFactory(jdbcUrl); @@ -175,7 +178,7 @@ private PreparedStatement prepareInsertStatement(SinkRow row) { } try { var preparedStmt = insertPreparedStmt; - jdbcDialect.bindInsertIntoStatement(preparedStmt, conn, getTableSchema(), row); + jdbcDialect.bindInsertIntoStatement(preparedStmt, conn, tableSchema, row); preparedStmt.addBatch(); return preparedStmt; } catch (SQLException e) { @@ -192,7 +195,7 @@ private PreparedStatement prepareUpsertStatement(SinkRow row) { var preparedStmt = upsertPreparedStmt; switch (row.getOp()) { case INSERT: - jdbcDialect.bindUpsertStatement(preparedStmt, conn, getTableSchema(), row); + jdbcDialect.bindUpsertStatement(preparedStmt, conn, tableSchema, row); break; case UPDATE_INSERT: if (!updateFlag) { @@ -200,7 +203,7 @@ private PreparedStatement prepareUpsertStatement(SinkRow row) { .withDescription("an UPDATE_DELETE should precede an UPDATE_INSERT") .asRuntimeException(); } - jdbcDialect.bindUpsertStatement(preparedStmt, conn, getTableSchema(), row); + jdbcDialect.bindUpsertStatement(preparedStmt, conn, tableSchema, row); updateFlag = false; break; default: @@ -235,7 +238,7 @@ private PreparedStatement prepareDeleteStatement(SinkRow row) { try { int placeholderIdx = 1; for (String primaryKey : pkColumnNames) { - Object fromRow = getTableSchema().getFromRow(primaryKey, row); + Object fromRow = tableSchema.getFromRow(primaryKey, row); deletePreparedStmt.setObject(placeholderIdx++, fromRow); } deletePreparedStmt.addBatch(); @@ -249,13 +252,15 @@ private PreparedStatement prepareDeleteStatement(SinkRow row) { } @Override - public void write(Iterator rows) { + public void beginEpoch(long epoch) {} + + @Override + public boolean write(Iterable rows) { PreparedStatement deleteStatement = null; PreparedStatement upsertStatement = null; PreparedStatement insertStatement = null; - while (rows.hasNext()) { - SinkRow row = rows.next(); + for (SinkRow row : rows) { if (row.getOp() == Data.Op.UPDATE_DELETE) { updateFlag = true; continue; @@ -285,6 +290,7 @@ public void write(Iterator rows) { String.format(ERROR_REPORT_TEMPLATE, e.getSQLState(), e.getMessage())) .asRuntimeException(); } + return true; } private void executeStatement(PreparedStatement stmt) throws SQLException { @@ -297,13 +303,14 @@ private void executeStatement(PreparedStatement stmt) throws SQLException { } @Override - public void sync() { + public Optional barrier(boolean isCheckpoint) { if (updateFlag) { throw Status.FAILED_PRECONDITION .withDescription( "expected UPDATE_INSERT to complete an UPDATE operation, got `sync`") .asRuntimeException(); } + return Optional.empty(); } @Override diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java index 39a86772d19e5..e4d009d32b1a0 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java @@ -19,7 +19,6 @@ import com.risingwave.connector.api.TableSchema; import com.risingwave.connector.api.sink.SinkFactory; import com.risingwave.connector.api.sink.SinkWriter; -import com.risingwave.connector.api.sink.SinkWriterV1; import com.risingwave.proto.Catalog.SinkType; import io.grpc.Status; import java.sql.*; @@ -40,7 +39,7 @@ public class JDBCSinkFactory implements SinkFactory { public SinkWriter createWriter(TableSchema tableSchema, Map tableProperties) { ObjectMapper mapper = new ObjectMapper(); JDBCSinkConfig config = mapper.convertValue(tableProperties, JDBCSinkConfig.class); - return new SinkWriterV1.Adapter(new JDBCSink(config, tableSchema)); + return new JDBCSink(config, tableSchema); } @Override diff --git a/proto/connector_service.proto b/proto/connector_service.proto index ed1a774623d92..db8b5a4e7e973 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -88,9 +88,15 @@ message SinkWriterStreamResponse { SinkMetadata metadata = 2; } + message BatchWrittenResponse { + uint64 epoch = 1; + uint64 batch_id = 2; + } + oneof response { StartResponse start = 1; CommitResponse commit = 2; + BatchWrittenResponse batch = 3; } } diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index f3de717a9596f..1c827309f30bb 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{HashMap, VecDeque}; use std::marker::PhantomData; use std::ops::Deref; use std::pin::pin; @@ -21,13 +21,14 @@ use std::time::Instant; use anyhow::anyhow; use async_trait::async_trait; use futures::future::select; -use futures::{StreamExt, TryFutureExt, TryStreamExt}; +use futures::{StreamExt, TryStreamExt}; use itertools::Itertools; use jni::JavaVM; use prost::Message; use risingwave_common::array::StreamChunk; use risingwave_common::error::anyhow_error; use risingwave_common::types::DataType; +use risingwave_common::util::drop_either_future; use risingwave_jni_core::jvm_runtime::JVM; use risingwave_jni_core::{call_static_method, gen_class_name, JniReceiverType, JniSenderType}; use risingwave_pb::connector_service::sink_coordinator_stream_request::StartCoordinator; @@ -51,7 +52,9 @@ use tokio::sync::mpsc; use tokio::sync::mpsc::{unbounded_channel, Receiver, Sender}; use tokio::task::spawn_blocking; use tokio_stream::wrappers::ReceiverStream; +use tracing::warn; +use crate::sink::catalog::desc::SinkDesc; use crate::sink::coordinate::CoordinatedSinkWriter; use crate::sink::log_store::{LogReader, LogStoreReadItem, TruncateOffset}; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; @@ -64,28 +67,54 @@ use crate::ConnectorParams; macro_rules! def_remote_sink { () => { def_remote_sink! { - { ElasticSearch, ElasticSearchSink, "elasticsearch" }, - { Cassandra, CassandraSink, "cassandra" }, - { Jdbc, JdbcSink, "jdbc" }, + { ElasticSearch, ElasticSearchSink, "elasticsearch" } + { Cassandra, CassandraSink, "cassandra" } + { Jdbc, JdbcSink, "jdbc", |desc| { + desc.sink_type.is_append_only() + } } { DeltaLake, DeltaLakeSink, "deltalake" } } }; - ($({ $variant_name:ident, $sink_type_name:ident, $sink_name:expr }),*) => { - $( - #[derive(Debug)] - pub struct $variant_name; - impl RemoteSinkTrait for $variant_name { - const SINK_NAME: &'static str = $sink_name; + () => {}; + ({ $variant_name:ident, $sink_type_name:ident, $sink_name:expr }) => { + #[derive(Debug)] + pub struct $variant_name; + impl RemoteSinkTrait for $variant_name { + const SINK_NAME: &'static str = $sink_name; + } + pub type $sink_type_name = RemoteSink<$variant_name>; + }; + ({ $variant_name:ident, $sink_type_name:ident, $sink_name:expr, |$desc:ident| $body:expr }) => { + #[derive(Debug)] + pub struct $variant_name; + impl RemoteSinkTrait for $variant_name { + const SINK_NAME: &'static str = $sink_name; + fn default_sink_decouple($desc: &SinkDesc) -> bool { + $body } - pub type $sink_type_name = RemoteSink<$variant_name>; - )* + } + pub type $sink_type_name = RemoteSink<$variant_name>; }; + ({ $($first:tt)+ } $({$($rest:tt)+})*) => { + def_remote_sink! { + {$($first)+} + } + def_remote_sink! { + $({$($rest)+})* + } + }; + ($($invalid:tt)*) => { + compile_error! {concat! {"invalid `", stringify!{$($invalid)*}, "`"}} + } } def_remote_sink!(); pub trait RemoteSinkTrait: Send + Sync + 'static { const SINK_NAME: &'static str; + fn default_sink_decouple(_desc: &SinkDesc) -> bool { + false + } } #[derive(Debug)] @@ -111,6 +140,10 @@ impl Sink for RemoteSink { const SINK_NAME: &'static str = R::SINK_NAME; + fn default_sink_decouple(desc: &SinkDesc) -> bool { + R::default_sink_decouple(desc) + } + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { RemoteLogSinker::new(self.param.clone(), writer_param).await } @@ -228,7 +261,7 @@ impl LogSinker for RemoteLogSinker { let (response_tx, mut response_rx) = unbounded_channel(); - let poll_response_stream = pin!(async move { + let poll_response_stream = async move { loop { let result = response_err_stream_rx.stream.try_next().await; match result { @@ -241,80 +274,164 @@ impl LogSinker for RemoteLogSinker { Err(e) => return Err(SinkError::Remote(anyhow!(e))), } } - }); - - let poll_consume_log_and_sink = pin!(async move { - let mut prev_offset: Option = None; + }; + let poll_consume_log_and_sink = async move { log_reader.init().await?; - loop { - let (epoch, item): (u64, LogStoreReadItem) = - log_reader.next_item().map_err(SinkError::Internal).await?; + async fn truncate_matched_offset( + queue: &mut VecDeque<(TruncateOffset, Option)>, + persisted_offset: TruncateOffset, + log_reader: &mut impl LogReader, + metrics: &SinkMetrics, + ) -> Result<()> { + while let Some((sent_offset, _)) = queue.front() && sent_offset < &persisted_offset { + queue.pop_front(); + } - match &prev_offset { - Some(TruncateOffset::Barrier { .. }) | None => { - // TODO: this start epoch is actually unnecessary - request_tx.start_epoch(epoch).await?; - } - _ => {} + let (sent_offset, start_time) = queue.pop_front().ok_or_else(|| { + anyhow!("get unsent offset {:?} in response", persisted_offset) + })?; + if sent_offset != persisted_offset { + return Err(anyhow!( + "new response offset {:?} not match the buffer offset {:?}", + persisted_offset, + sent_offset + ) + .into()); + } + + if let (TruncateOffset::Barrier { .. }, Some(start_time)) = + (persisted_offset, start_time) + { + metrics + .sink_commit_duration_metrics + .observe(start_time.elapsed().as_millis() as f64); } - match item { - LogStoreReadItem::StreamChunk { chunk, chunk_id } => { - let offset = TruncateOffset::Chunk { epoch, chunk_id }; - if let Some(prev_offset) = &prev_offset { - prev_offset.check_next_offset(offset)?; + log_reader.truncate(persisted_offset).await?; + Ok(()) + } + + let mut prev_offset: Option = None; + // Push from back and pop from front + let mut sent_offset_queue: VecDeque<(TruncateOffset, Option)> = + VecDeque::new(); + + loop { + let either_result: futures::future::Either< + Option, + anyhow::Result<(u64, LogStoreReadItem)>, + > = drop_either_future( + select(pin!(response_rx.recv()), pin!(log_reader.next_item())).await, + ); + match either_result { + futures::future::Either::Left(opt) => { + let response = opt.ok_or_else(|| anyhow!("end of response stream"))?; + match response { + SinkWriterStreamResponse { + response: + Some(sink_writer_stream_response::Response::Batch( + sink_writer_stream_response::BatchWrittenResponse { + epoch, + batch_id, + }, + )), + } => { + truncate_matched_offset( + &mut sent_offset_queue, + TruncateOffset::Chunk { + epoch, + chunk_id: batch_id as _, + }, + &mut log_reader, + &sink_metrics, + ) + .await?; + } + SinkWriterStreamResponse { + response: + Some(sink_writer_stream_response::Response::Commit( + sink_writer_stream_response::CommitResponse { + epoch, + metadata, + }, + )), + } => { + if let Some(metadata) = metadata { + warn!("get unexpected non-empty metadata: {:?}", metadata); + } + truncate_matched_offset( + &mut sent_offset_queue, + TruncateOffset::Barrier { epoch }, + &mut log_reader, + &sink_metrics, + ) + .await?; + } + response => { + return Err(SinkError::Remote(anyhow!( + "get unexpected response: {:?}", + response + ))); + } } - let cardinality = chunk.cardinality(); - sink_metrics - .connector_sink_rows_received - .inc_by(cardinality as _); - - let payload = build_chunk_payload(chunk); - request_tx - .write_batch(epoch, chunk_id as u64, payload) - .await?; - prev_offset = Some(offset); } - LogStoreReadItem::Barrier { is_checkpoint } => { - let offset = TruncateOffset::Barrier { epoch }; - if let Some(prev_offset) = &prev_offset { - prev_offset.check_next_offset(offset)?; + futures::future::Either::Right(result) => { + let (epoch, item): (u64, LogStoreReadItem) = result?; + + match &prev_offset { + Some(TruncateOffset::Barrier { .. }) | None => { + // TODO: this start epoch is actually unnecessary + request_tx.start_epoch(epoch).await?; + } + _ => {} } - if is_checkpoint { - let start_time = Instant::now(); - request_tx.barrier(epoch, true).await?; - match response_rx.recv().await.ok_or_else(|| { - SinkError::Remote(anyhow!("end of response stream")) - })? { - SinkWriterStreamResponse { - response: Some(sink_writer_stream_response::Response::Commit(_)), - } => {} - response => { - return Err(SinkError::Remote(anyhow!( - "expected commit response, but get {:?}", - response - ))); + + match item { + LogStoreReadItem::StreamChunk { chunk, chunk_id } => { + let offset = TruncateOffset::Chunk { epoch, chunk_id }; + if let Some(prev_offset) = &prev_offset { + prev_offset.check_next_offset(offset)?; } - }; - sink_metrics - .sink_commit_duration_metrics - .observe(start_time.elapsed().as_millis() as f64); - log_reader - .truncate(TruncateOffset::Barrier { epoch }) - .await?; - } else { - request_tx.barrier(epoch, false).await?; + let cardinality = chunk.cardinality(); + sink_metrics + .connector_sink_rows_received + .inc_by(cardinality as _); + + let payload = build_chunk_payload(chunk); + request_tx + .write_batch(epoch, chunk_id as u64, payload) + .await?; + prev_offset = Some(offset); + sent_offset_queue + .push_back((TruncateOffset::Chunk { epoch, chunk_id }, None)); + } + LogStoreReadItem::Barrier { is_checkpoint } => { + let offset = TruncateOffset::Barrier { epoch }; + if let Some(prev_offset) = &prev_offset { + prev_offset.check_next_offset(offset)?; + } + let start_time = if is_checkpoint { + let start_time = Instant::now(); + request_tx.barrier(epoch, true).await?; + Some(start_time) + } else { + request_tx.barrier(epoch, false).await?; + None + }; + prev_offset = Some(offset); + sent_offset_queue + .push_back((TruncateOffset::Barrier { epoch }, start_time)); + } + LogStoreReadItem::UpdateVnodeBitmap(_) => {} } - prev_offset = Some(offset); } - LogStoreReadItem::UpdateVnodeBitmap(_) => {} } } - }); + }; - select(poll_response_stream, poll_consume_log_and_sink) + select(pin!(poll_response_stream), pin!(poll_consume_log_and_sink)) .await .factor_first() .0 From 8e231af93a8c4bd37f0ce933c5044a83ad51af67 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Wed, 8 Nov 2023 15:04:49 +0800 Subject: [PATCH 08/77] feat(pgwire): listen on unix socket (#13291) Signed-off-by: Bugen Zhao --- Cargo.lock | 2 + src/utils/pgwire/Cargo.toml | 2 + src/utils/pgwire/src/lib.rs | 1 + src/utils/pgwire/src/net.rs | 87 +++++++++++++++++++++++++++++ src/utils/pgwire/src/pg_server.rs | 91 ++++++++++++++++++++----------- src/workspace-hack/Cargo.toml | 2 +- 6 files changed, 151 insertions(+), 34 deletions(-) create mode 100644 src/utils/pgwire/src/net.rs diff --git a/Cargo.lock b/Cargo.lock index 4c8bf0942c393..8fd4e50b5e139 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5943,6 +5943,7 @@ name = "pgwire" version = "1.3.0-alpha" dependencies = [ "anyhow", + "auto_enums", "byteorder", "bytes", "futures", @@ -5952,6 +5953,7 @@ dependencies = [ "panic-message", "risingwave_common", "risingwave_sqlparser", + "tempfile", "thiserror", "tokio-openssl", "tokio-postgres", diff --git a/src/utils/pgwire/Cargo.toml b/src/utils/pgwire/Cargo.toml index c6d46e356518a..c7b6faf3c885d 100644 --- a/src/utils/pgwire/Cargo.toml +++ b/src/utils/pgwire/Cargo.toml @@ -16,6 +16,7 @@ normal = ["workspace-hack"] [dependencies] anyhow = { version = "1.0", default-features = false } +auto_enums = { version = "0.8", features = ["tokio1"] } byteorder = "1.5" bytes = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } @@ -33,6 +34,7 @@ tracing = "0.1" workspace-hack = { path = "../../workspace-hack" } [dev-dependencies] +tempfile = "3" tokio-postgres = "0.7" [lints] diff --git a/src/utils/pgwire/src/lib.rs b/src/utils/pgwire/src/lib.rs index 84a17d9907879..e7b487bb42c2b 100644 --- a/src/utils/pgwire/src/lib.rs +++ b/src/utils/pgwire/src/lib.rs @@ -22,6 +22,7 @@ pub mod error; pub mod error_or_notice; +pub mod net; pub mod pg_extended; pub mod pg_field_descriptor; pub mod pg_message; diff --git a/src/utils/pgwire/src/net.rs b/src/utils/pgwire/src/net.rs new file mode 100644 index 0000000000000..ce341dec3e742 --- /dev/null +++ b/src/utils/pgwire/src/net.rs @@ -0,0 +1,87 @@ +// 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 std::io; +use std::net::SocketAddr as IpSocketAddr; +#[cfg(madsim)] +use std::os::unix::net::SocketAddr as UnixSocketAddr; + +#[cfg(not(madsim))] +use tokio::net::unix::SocketAddr as UnixSocketAddr; +use tokio::net::{TcpListener, TcpStream, UnixListener, UnixStream}; + +/// A wrapper of either [`TcpListener`] or [`UnixListener`]. +pub(crate) enum Listener { + Tcp(TcpListener), + Unix(UnixListener), +} + +/// A wrapper of either [`TcpStream`] or [`UnixStream`]. +#[auto_enums::enum_derive(tokio1::AsyncRead, tokio1::AsyncWrite)] +pub(crate) enum Stream { + Tcp(TcpStream), + Unix(UnixStream), +} + +/// A wrapper of either [`std::net::SocketAddr`] or [`tokio::net::unix::SocketAddr`]. +pub(crate) enum Address { + Tcp(IpSocketAddr), + Unix(UnixSocketAddr), +} + +impl std::fmt::Display for Address { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Address::Tcp(addr) => addr.fmt(f), + Address::Unix(addr) => { + if let Some(path) = addr.as_pathname() { + path.display().fmt(f) + } else { + std::fmt::Debug::fmt(addr, f) + } + } + } + } +} + +impl Listener { + /// Creates a new [`Listener`] bound to the specified address. + /// + /// If the address starts with `unix:`, it will create a [`UnixListener`]. + /// Otherwise, it will create a [`TcpListener`]. + pub async fn bind(addr: &str) -> io::Result { + if let Some(path) = addr.strip_prefix("unix:") { + UnixListener::bind(path).map(Self::Unix) + } else { + TcpListener::bind(addr).await.map(Self::Tcp) + } + } + + /// Accepts a new incoming connection from this listener. + /// + /// Returns a tuple of the stream and the string representation of the peer address. + pub async fn accept(&self) -> io::Result<(Stream, Address)> { + match self { + Self::Tcp(listener) => { + let (stream, addr) = listener.accept().await?; + stream.set_nodelay(true)?; + Ok((Stream::Tcp(stream), Address::Tcp(addr))) + } + Self::Unix(listener) => { + let (stream, addr) = listener.accept().await?; + Ok((Stream::Unix(stream), Address::Unix(addr))) + } + } + } +} diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index ba52215e4d34a..2734ff857735e 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -18,13 +18,11 @@ use std::result::Result; use std::sync::Arc; use bytes::Bytes; -use futures::TryFutureExt; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::Statement; use tokio::io::{AsyncRead, AsyncWrite}; -use tokio::net::TcpListener; -use tracing::debug; +use crate::net::Listener; use crate::pg_field_descriptor::PgFieldDescriptor; use crate::pg_message::TransactionStatus; use crate::pg_protocol::{PgProtocol, TlsConfig}; @@ -130,53 +128,61 @@ impl UserAuthenticator { } } -/// Binds a Tcp listener at `addr`. Spawn a coroutine to serve every new connection. +/// Binds a Tcp or Unix listener at `addr`. Spawn a coroutine to serve every new connection. pub async fn pg_serve( addr: &str, session_mgr: Arc, - ssl_config: Option, + tls_config: Option, ) -> io::Result<()> { - let listener = TcpListener::bind(addr).await.unwrap(); - // accept connections and process them, spawning a new thread for each one - tracing::info!("Server Listening at {}", addr); + let listener = Listener::bind(addr).await?; + tracing::info!(addr, "server started"); + loop { - let session_mgr = session_mgr.clone(); let conn_ret = listener.accept().await; match conn_ret { Ok((stream, peer_addr)) => { - tracing::info!("New connection: {}", peer_addr); - stream.set_nodelay(true)?; - let ssl_config = ssl_config.clone(); - let fut = handle_connection(stream, session_mgr, ssl_config); - tokio::spawn(fut.inspect_err(|e| debug!("error handling connection: {e}"))); + tracing::info!(%peer_addr, "accept connection"); + tokio::spawn(handle_connection( + stream, + session_mgr.clone(), + tls_config.clone(), + )); } Err(e) => { - tracing::error!("Connection failure: {}", e); + tracing::error!( + error = &e as &dyn std::error::Error, + "failed to accept connection", + ); } } } } -#[tracing::instrument(level = "debug", skip_all)] -pub fn handle_connection( +pub async fn handle_connection( stream: S, session_mgr: Arc, tls_config: Option, -) -> impl Future> -where +) where S: AsyncWrite + AsyncRead + Unpin, SM: SessionManager, { let mut pg_proto = PgProtocol::new(stream, session_mgr, tls_config); - async { - loop { - let msg = pg_proto.read_message().await?; - tracing::trace!("Received message: {:?}", msg); - let ret = pg_proto.process(msg).await; - if ret { - return Ok(()); + loop { + let msg = match pg_proto.read_message().await { + Ok(msg) => msg, + Err(e) => { + tracing::error!( + error = &e as &dyn std::error::Error, + "error when reading message" + ); + break; } + }; + tracing::trace!("Received message: {:?}", msg); + let ret = pg_proto.process(msg).await; + if ret { + break; } } } @@ -325,17 +331,17 @@ mod tests { } } - #[tokio::test] - async fn test_query() { + async fn do_test_query(bind_addr: impl Into, pg_config: impl Into) { + let bind_addr = bind_addr.into(); + let pg_config = pg_config.into(); + let session_mgr = Arc::new(MockSessionManager {}); - tokio::spawn(async move { pg_serve("127.0.0.1:10000", session_mgr, None).await }); + tokio::spawn(async move { pg_serve(&bind_addr, session_mgr, None).await }); // wait for server to start - tokio::time::sleep(std::time::Duration::from_millis(10)).await; + tokio::time::sleep(std::time::Duration::from_millis(100)).await; // Connect to the database. - let (client, connection) = tokio_postgres::connect("host=localhost port=10000", NoTls) - .await - .unwrap(); + let (client, connection) = tokio_postgres::connect(&pg_config, NoTls).await.unwrap(); // The connection object performs the actual communication with the database, // so spawn it off to run on its own. @@ -358,4 +364,23 @@ mod tests { .expect("Error executing query"); assert_eq!(rows.len(), 1); } + + #[tokio::test] + async fn test_query_tcp() { + do_test_query("127.0.0.1:10000", "host=localhost port=10000").await; + } + + #[cfg(not(madsim))] + #[tokio::test] + async fn test_query_unix() { + let port: i16 = 10000; + let dir = tempfile::TempDir::new().unwrap(); + let sock = dir.path().join(format!(".s.PGSQL.{port}")); + + do_test_query( + format!("unix:{}", sock.to_str().unwrap()), + format!("host={} port={}", dir.path().to_str().unwrap(), port), + ) + .await; + } } diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 0edc6c7bb5ec6..fe483dbc7b1a0 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -146,7 +146,7 @@ whoami = { version = "1" } ahash = { version = "0.8" } allocator-api2 = { version = "0.2", default-features = false, features = ["alloc", "nightly"] } anyhow = { version = "1", features = ["backtrace"] } -auto_enums = { version = "0.8", features = ["futures03"] } +auto_enums = { version = "0.8", features = ["futures03", "tokio1"] } bitflags = { version = "2", default-features = false, features = ["serde", "std"] } bytes = { version = "1", features = ["serde"] } cc = { version = "1", default-features = false, features = ["parallel"] } From 89f77f53586a7b73ca94a9c12ff9d44b1b9e4b14 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 8 Nov 2023 16:12:34 +0800 Subject: [PATCH 09/77] fix(jdbc-sink): Handle JDBC connection error by recreating the connection (#13279) --- .../connector/sink/jdbc/JDBCSinkTest.java | 22 +- .../com/risingwave/connector/JDBCSink.java | 396 ++++++++++-------- .../com/risingwave/connector/JdbcUtils.java | 20 + 3 files changed, 251 insertions(+), 187 deletions(-) diff --git a/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/jdbc/JDBCSinkTest.java b/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/jdbc/JDBCSinkTest.java index 918ea46f9e49f..537b7a415ac19 100644 --- a/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/jdbc/JDBCSinkTest.java +++ b/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/jdbc/JDBCSinkTest.java @@ -25,7 +25,7 @@ import com.risingwave.proto.Data.DataType.TypeName; import com.risingwave.proto.Data.Op; import java.sql.*; -import java.util.Arrays; +import java.util.List; import org.junit.Test; import org.testcontainers.containers.JdbcDatabaseContainer; import org.testcontainers.containers.MySQLContainer; @@ -81,10 +81,10 @@ static void testJDBCSync(JdbcDatabaseContainer container, TestType testType) new JDBCSinkConfig(container.getJdbcUrl(), tableName, "upsert"), getTestTableSchema()); assertEquals(tableName, sink.getTableName()); - Connection conn = sink.getConn(); + Connection conn = DriverManager.getConnection(container.getJdbcUrl()); sink.write( - Arrays.asList( + List.of( new ArraySinkRow( Op.INSERT, 1, @@ -106,7 +106,7 @@ static void testJDBCSync(JdbcDatabaseContainer container, TestType testType) } sink.write( - Arrays.asList( + List.of( new ArraySinkRow( Op.INSERT, 2, @@ -125,6 +125,7 @@ static void testJDBCSync(JdbcDatabaseContainer container, TestType testType) assertEquals(2, count); } stmt.close(); + conn.close(); sink.barrier(true); sink.drop(); @@ -140,11 +141,11 @@ static void testJDBCWrite(JdbcDatabaseContainer container, TestType testType) new JDBCSinkConfig(container.getJdbcUrl(), tableName, "upsert"), getTestTableSchema()); assertEquals(tableName, sink.getTableName()); - Connection conn = sink.getConn(); + Connection conn = DriverManager.getConnection(container.getJdbcUrl()); Statement stmt = conn.createStatement(); sink.write( - Arrays.asList( + List.of( new ArraySinkRow( Op.INSERT, 1, @@ -171,7 +172,7 @@ static void testJDBCWrite(JdbcDatabaseContainer container, TestType testType) } sink.write( - Arrays.asList( + List.of( new ArraySinkRow( Op.UPDATE_DELETE, 1, @@ -218,6 +219,7 @@ static void testJDBCWrite(JdbcDatabaseContainer container, TestType testType) sink.barrier(true); stmt.close(); + conn.close(); } static void testJDBCDrop(JdbcDatabaseContainer container, TestType testType) @@ -232,11 +234,7 @@ static void testJDBCDrop(JdbcDatabaseContainer container, TestType testType) assertEquals(tableName, sink.getTableName()); Connection conn = sink.getConn(); sink.drop(); - try { - assertTrue(conn.isClosed()); - } catch (SQLException e) { - fail(String.valueOf(e)); - } + assertTrue(conn.isClosed()); } @Test diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java index 220caca4d13fe..cda733e38ff5f 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java @@ -31,7 +31,8 @@ public class JDBCSink implements SinkWriter { private final JdbcDialect jdbcDialect; private final JDBCSinkConfig config; - private final Connection conn; + private Connection conn; + private JdbcStatements jdbcStatements; private final List pkColumnNames; private final TableSchema tableSchema; @@ -39,10 +40,6 @@ public class JDBCSink implements SinkWriter { public static final String JDBC_COLUMN_NAME_KEY = "COLUMN_NAME"; public static final String JDBC_DATA_TYPE_KEY = "DATA_TYPE"; - private PreparedStatement insertPreparedStmt; - private PreparedStatement upsertPreparedStmt; - private PreparedStatement deletePreparedStmt; - private boolean updateFlag = false; private static final Logger LOG = LoggerFactory.getLogger(JDBCSink.class); @@ -54,13 +51,13 @@ public JDBCSink(JDBCSinkConfig config, TableSchema tableSchema) { var factory = JdbcUtils.getDialectFactory(jdbcUrl); this.config = config; try { - this.conn = DriverManager.getConnection(config.getJdbcUrl()); + conn = JdbcUtils.getConnection(config.getJdbcUrl()); + // Retrieve primary keys and column type mappings from the database this.pkColumnNames = getPkColumnNames(conn, config.getTableName(), config.getSchemaName()); // column name -> java.sql.Types Map columnTypeMapping = getColumnTypeMapping(conn, config.getTableName(), config.getSchemaName()); - // create an array that each slot corresponding to each column in TableSchema var columnSqlTypes = new int[tableSchema.getNumColumns()]; for (int columnIdx = 0; columnIdx < tableSchema.getNumColumns(); columnIdx++) { @@ -76,52 +73,12 @@ public JDBCSink(JDBCSinkConfig config, TableSchema tableSchema) { .withDescription("Unsupported jdbc url: " + jdbcUrl) .asRuntimeException(); } - - // disable auto commit can improve performance - this.conn.setAutoCommit(false); - // explicitly set isolation level to RC - this.conn.setTransactionIsolation(Connection.TRANSACTION_READ_COMMITTED); - LOG.info( "JDBC connection: autoCommit = {}, trxn = {}", conn.getAutoCommit(), conn.getTransactionIsolation()); - } catch (SQLException e) { - throw Status.INTERNAL - .withDescription( - String.format(ERROR_REPORT_TEMPLATE, e.getSQLState(), e.getMessage())) - .asRuntimeException(); - } - try { - var schemaTableName = - jdbcDialect.createSchemaTableName( - config.getSchemaName(), config.getTableName()); - if (config.isUpsertSink()) { - var upsertSql = - jdbcDialect.getUpsertStatement( - schemaTableName, - List.of(tableSchema.getColumnNames()), - pkColumnNames); - // MySQL and Postgres have upsert SQL - if (upsertSql.isEmpty()) { - throw Status.FAILED_PRECONDITION - .withDescription("Failed to get upsert SQL") - .asRuntimeException(); - } - this.upsertPreparedStmt = - conn.prepareStatement(upsertSql.get(), Statement.RETURN_GENERATED_KEYS); - // upsert sink will handle DELETE events - var deleteSql = jdbcDialect.getDeleteStatement(schemaTableName, pkColumnNames); - this.deletePreparedStmt = - conn.prepareStatement(deleteSql, Statement.RETURN_GENERATED_KEYS); - } else { - var insertSql = - jdbcDialect.getInsertIntoStatement( - schemaTableName, List.of(tableSchema.getColumnNames())); - this.insertPreparedStmt = - conn.prepareStatement(insertSql, Statement.RETURN_GENERATED_KEYS); - } + jdbcStatements = new JdbcStatements(conn); } catch (SQLException e) { throw Status.INTERNAL .withDescription( @@ -170,138 +127,244 @@ private static List getPkColumnNames( return pkColumnNames; } - private PreparedStatement prepareInsertStatement(SinkRow row) { - if (row.getOp() != Data.Op.INSERT) { - throw Status.FAILED_PRECONDITION - .withDescription("unexpected op type: " + row.getOp()) - .asRuntimeException(); - } - try { - var preparedStmt = insertPreparedStmt; - jdbcDialect.bindInsertIntoStatement(preparedStmt, conn, tableSchema, row); - preparedStmt.addBatch(); - return preparedStmt; - } catch (SQLException e) { - throw io.grpc.Status.INTERNAL - .withDescription( - String.format(ERROR_REPORT_TEMPLATE, e.getSQLState(), e.getMessage())) - .withCause(e) - .asRuntimeException(); - } - } + @Override + public boolean write(Iterable rows) { + final int maxRetryCount = 4; + int retryCount = 0; + while (true) { + try { + // fill prepare statements with parameters + for (SinkRow row : rows) { + if (row.getOp() == Data.Op.UPDATE_DELETE) { + updateFlag = true; + continue; + } + if (config.isUpsertSink()) { + if (row.getOp() == Data.Op.DELETE) { + jdbcStatements.prepareDelete(row); + } else { + jdbcStatements.prepareUpsert(row); + } + } else { + jdbcStatements.prepareInsert(row); + } + } + // Execute staging statements after all rows are prepared. + jdbcStatements.execute(); + break; + } catch (SQLException e) { + LOG.error("Failed to execute JDBC statements, retried {} times", retryCount, e); + if (++retryCount > maxRetryCount) { + throw Status.INTERNAL + .withDescription( + "Failed to execute JDBC statements and exceeded max retry times") + .withCause(e) + .asRuntimeException(); + } - private PreparedStatement prepareUpsertStatement(SinkRow row) { - try { - var preparedStmt = upsertPreparedStmt; - switch (row.getOp()) { - case INSERT: - jdbcDialect.bindUpsertStatement(preparedStmt, conn, tableSchema, row); - break; - case UPDATE_INSERT: - if (!updateFlag) { - throw Status.FAILED_PRECONDITION - .withDescription("an UPDATE_DELETE should precede an UPDATE_INSERT") + try { + if (!conn.isValid(10)) { // 10 seconds timeout + LOG.info("Recreate the JDBC connection due to connection broken"); + // close the statements and connection first + jdbcStatements.close(); + conn.close(); + + // create a new connection if the current connection is invalid + conn = JdbcUtils.getConnection(config.getJdbcUrl()); + // reset the flag since we will retry to prepare the batch again + updateFlag = false; + jdbcStatements = new JdbcStatements(conn); + } else { + throw io.grpc.Status.INTERNAL + .withDescription( + String.format( + ERROR_REPORT_TEMPLATE, + e.getSQLState(), + e.getMessage())) .asRuntimeException(); } - jdbcDialect.bindUpsertStatement(preparedStmt, conn, tableSchema, row); - updateFlag = false; - break; - default: - throw Status.FAILED_PRECONDITION - .withDescription("unexpected op type: " + row.getOp()) + } catch (SQLException ex) { + LOG.error( + "Failed to create a new JDBC connection, retried {} times", + retryCount, + ex); + throw io.grpc.Status.INTERNAL + .withDescription( + String.format( + ERROR_REPORT_TEMPLATE, + ex.getSQLState(), + ex.getMessage())) .asRuntimeException(); + } } - preparedStmt.addBatch(); - return preparedStmt; - } catch (SQLException e) { - throw io.grpc.Status.INTERNAL - .withDescription( - String.format(ERROR_REPORT_TEMPLATE, e.getSQLState(), e.getMessage())) - .withCause(e) - .asRuntimeException(); } + return true; } - private PreparedStatement prepareDeleteStatement(SinkRow row) { - if (!config.isUpsertSink()) { - throw Status.FAILED_PRECONDITION - .withDescription("Non-upsert sink cannot handle DELETE event") - .asRuntimeException(); - } - if (pkColumnNames.isEmpty()) { - throw Status.INTERNAL - .withDescription( - "downstream jdbc table should have primary key to handle DELETE event") - .asRuntimeException(); - } + /** + * A wrapper class for JDBC prepared statements We create this object one time and reuse it + * across multiple batches if only the JDBC connection is valid. + */ + class JdbcStatements implements AutoCloseable { + private PreparedStatement deleteStatement; + private PreparedStatement upsertStatement; + private PreparedStatement insertStatement; - try { - int placeholderIdx = 1; - for (String primaryKey : pkColumnNames) { - Object fromRow = tableSchema.getFromRow(primaryKey, row); - deletePreparedStmt.setObject(placeholderIdx++, fromRow); + private final Connection conn; + + public JdbcStatements(Connection conn) throws SQLException { + this.conn = conn; + var schemaTableName = + jdbcDialect.createSchemaTableName( + config.getSchemaName(), config.getTableName()); + + if (config.isUpsertSink()) { + var upsertSql = + jdbcDialect.getUpsertStatement( + schemaTableName, + List.of(tableSchema.getColumnNames()), + pkColumnNames); + // MySQL and Postgres have upsert SQL + if (upsertSql.isEmpty()) { + throw Status.FAILED_PRECONDITION + .withDescription("Failed to get upsert SQL") + .asRuntimeException(); + } + + this.upsertStatement = + conn.prepareStatement(upsertSql.get(), Statement.RETURN_GENERATED_KEYS); + // upsert sink will handle DELETE events + var deleteSql = jdbcDialect.getDeleteStatement(schemaTableName, pkColumnNames); + this.deleteStatement = + conn.prepareStatement(deleteSql, Statement.RETURN_GENERATED_KEYS); + } else { + var insertSql = + jdbcDialect.getInsertIntoStatement( + schemaTableName, List.of(tableSchema.getColumnNames())); + this.insertStatement = + conn.prepareStatement(insertSql, Statement.RETURN_GENERATED_KEYS); } - deletePreparedStmt.addBatch(); - return deletePreparedStmt; - } catch (SQLException e) { - throw Status.INTERNAL - .withDescription( - String.format(ERROR_REPORT_TEMPLATE, e.getSQLState(), e.getMessage())) - .asRuntimeException(); } - } - @Override - public void beginEpoch(long epoch) {} + public void prepareUpsert(SinkRow row) { + try { + switch (row.getOp()) { + case INSERT: + jdbcDialect.bindUpsertStatement(upsertStatement, conn, tableSchema, row); + break; + case UPDATE_INSERT: + if (!updateFlag) { + throw Status.FAILED_PRECONDITION + .withDescription( + "an UPDATE_DELETE should precede an UPDATE_INSERT") + .asRuntimeException(); + } + jdbcDialect.bindUpsertStatement(upsertStatement, conn, tableSchema, row); + updateFlag = false; + break; + default: + throw Status.FAILED_PRECONDITION + .withDescription("unexpected op type: " + row.getOp()) + .asRuntimeException(); + } + upsertStatement.addBatch(); + } catch (SQLException e) { + throw io.grpc.Status.INTERNAL + .withDescription( + String.format( + ERROR_REPORT_TEMPLATE, e.getSQLState(), e.getMessage())) + .withCause(e) + .asRuntimeException(); + } + } - @Override - public boolean write(Iterable rows) { - PreparedStatement deleteStatement = null; - PreparedStatement upsertStatement = null; - PreparedStatement insertStatement = null; - - for (SinkRow row : rows) { - if (row.getOp() == Data.Op.UPDATE_DELETE) { - updateFlag = true; - continue; + public void prepareDelete(SinkRow row) { + if (!config.isUpsertSink()) { + throw Status.FAILED_PRECONDITION + .withDescription("Non-upsert sink cannot handle DELETE event") + .asRuntimeException(); } - if (config.isUpsertSink()) { - if (row.getOp() == Data.Op.DELETE) { - deleteStatement = prepareDeleteStatement(row); - } else { - upsertStatement = prepareUpsertStatement(row); + if (pkColumnNames.isEmpty()) { + throw Status.INTERNAL + .withDescription( + "downstream jdbc table should have primary key to handle DELETE event") + .asRuntimeException(); + } + try { + int placeholderIdx = 1; + for (String primaryKey : pkColumnNames) { + Object fromRow = tableSchema.getFromRow(primaryKey, row); + deleteStatement.setObject(placeholderIdx++, fromRow); } - } else { - insertStatement = prepareInsertStatement(row); + deleteStatement.addBatch(); + } catch (SQLException e) { + throw Status.INTERNAL + .withDescription( + String.format( + ERROR_REPORT_TEMPLATE, e.getSQLState(), e.getMessage())) + .asRuntimeException(); } } - try { - // Execute staging statements after all rows are prepared. + public void prepareInsert(SinkRow row) { + if (row.getOp() != Data.Op.INSERT) { + throw Status.FAILED_PRECONDITION + .withDescription("unexpected op type: " + row.getOp()) + .asRuntimeException(); + } + try { + jdbcDialect.bindInsertIntoStatement(insertStatement, conn, tableSchema, row); + insertStatement.addBatch(); + } catch (SQLException e) { + throw io.grpc.Status.INTERNAL + .withDescription( + String.format( + ERROR_REPORT_TEMPLATE, e.getSQLState(), e.getMessage())) + .withCause(e) + .asRuntimeException(); + } + } + + public void execute() throws SQLException { // We execute DELETE statement before to avoid accidentally deletion. - executeStatement(deleteStatement); - executeStatement(upsertStatement); - executeStatement(insertStatement); + executeStatement(this.deleteStatement); + executeStatement(this.upsertStatement); + executeStatement(this.insertStatement); - conn.commit(); - } catch (SQLException e) { - throw io.grpc.Status.INTERNAL - .withDescription( - String.format(ERROR_REPORT_TEMPLATE, e.getSQLState(), e.getMessage())) - .asRuntimeException(); + this.conn.commit(); + } + + @Override + public void close() { + closeStatement(this.deleteStatement); + closeStatement(this.upsertStatement); + closeStatement(this.insertStatement); + // we don't close the connection here, because we don't own it + } + + private void executeStatement(PreparedStatement stmt) throws SQLException { + if (stmt == null) { + return; + } + LOG.debug("Executing statement: {}", stmt); + stmt.executeBatch(); + stmt.clearParameters(); } - return true; - } - private void executeStatement(PreparedStatement stmt) throws SQLException { - if (stmt == null) { - return; + private void closeStatement(PreparedStatement stmt) { + try { + if (stmt != null) { + stmt.close(); + } + } catch (SQLException e) { + LOG.error("unable to close the prepared stmt", e); + } } - LOG.debug("Executing statement: {}", stmt); - stmt.executeBatch(); - stmt.clearParameters(); } + @Override + public void beginEpoch(long epoch) {} + @Override public Optional barrier(boolean isCheckpoint) { if (updateFlag) { @@ -315,27 +378,10 @@ public Optional barrier(boolean isCheckpoint @Override public void drop() { - try { - if (upsertPreparedStmt != null) { - upsertPreparedStmt.close(); - } - } catch (SQLException e) { - LOG.error("unable to close upsert stmt: %s", e); - } - try { - if (deletePreparedStmt != null) { - deletePreparedStmt.close(); - } - } catch (SQLException e) { - LOG.error("unable to close delete stmt: %s", e); - } - try { - if (insertPreparedStmt != null) { - insertPreparedStmt.close(); - } - } catch (SQLException e) { - LOG.error("unable to close insert stmt: %s", e); + if (jdbcStatements != null) { + jdbcStatements.close(); } + try { if (conn != null) { conn.close(); diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JdbcUtils.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JdbcUtils.java index 3401389bb7db5..95e51563a9301 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JdbcUtils.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JdbcUtils.java @@ -17,7 +17,11 @@ import com.risingwave.connector.jdbc.JdbcDialectFactory; import com.risingwave.connector.jdbc.MySqlDialectFactory; import com.risingwave.connector.jdbc.PostgresDialectFactory; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; import java.util.Optional; +import java.util.Properties; public abstract class JdbcUtils { @@ -30,4 +34,20 @@ public static Optional getDialectFactory(String jdbcUrl) { return Optional.empty(); } } + + /** The connection returned by this method is *not* autoCommit */ + public static Connection getConnection(String jdbcUrl) throws SQLException { + var props = new Properties(); + // enable TCP keep alive to avoid connection closed by server + // both MySQL and PG support this property + // https://jdbc.postgresql.org/documentation/use/ + // https://dev.mysql.com/doc/connectors/en/connector-j-connp-props-networking.html#cj-conn-prop_tcpKeepAlive + props.setProperty("tcpKeepAlive", "true"); + var conn = DriverManager.getConnection(jdbcUrl, props); + // disable auto commit can improve performance + conn.setAutoCommit(false); + // explicitly set isolation level to RC + conn.setTransactionIsolation(Connection.TRANSACTION_READ_COMMITTED); + return conn; + } } From 285296d80361ba0d87c8680bef3d24d41c50d6ba Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Wed, 8 Nov 2023 16:13:47 +0800 Subject: [PATCH 10/77] refactor: rename "chain" node (#13242) --- dashboard/lib/streamPlan/parser.js | 4 +- proto/stream_plan.proto | 49 ++--- proto/stream_service.proto | 2 +- src/common/src/catalog/external_table.rs | 2 +- src/common/src/util/stream_graph_visitor.rs | 6 +- .../tests/testdata/output/agg.yaml | 4 +- .../tests/testdata/output/ch_benchmark.yaml | 183 +++++++++--------- .../testdata/output/distribution_derive.yaml | 69 +++---- .../testdata/output/emit_on_window_close.yaml | 2 +- .../tests/testdata/output/except.yaml | 8 +- .../tests/testdata/output/intersect.yaml | 8 +- .../tests/testdata/output/join.yaml | 6 +- .../tests/testdata/output/nexmark.yaml | 122 ++++++------ .../testdata/output/over_window_function.yaml | 3 +- .../tests/testdata/output/share.yaml | 2 +- .../testdata/output/stream_dist_agg.yaml | 68 +++---- .../testdata/output/temporal_filter.yaml | 3 +- .../tests/testdata/output/tpch.yaml | 179 ++++++++--------- .../tests/testdata/output/union.yaml | 54 +++--- .../system_catalog/rw_catalog/rw_fragments.rs | 4 +- .../src/optimizer/plan_node/logical_join.rs | 4 +- .../src/optimizer/plan_node/logical_scan.rs | 9 +- .../optimizer/plan_node/stream_table_scan.rs | 50 ++--- .../optimizer/rule/index_delta_join_rule.rs | 15 +- src/frontend/src/stream_fragmenter/mod.rs | 6 +- .../src/utils/stream_graph_formatter.rs | 2 +- src/meta/src/barrier/command.rs | 6 +- src/meta/src/barrier/notifier.rs | 2 +- src/meta/src/barrier/progress.rs | 42 ++-- src/meta/src/manager/catalog/fragment.rs | 12 +- src/meta/src/model/stream.rs | 10 +- src/meta/src/rpc/ddl_controller.rs | 2 +- src/meta/src/stream/stream_graph/actor.rs | 18 +- src/meta/src/stream/stream_graph/fragment.rs | 22 +-- src/stream/src/from_proto/mod.rs | 6 +- .../from_proto/{chain.rs => stream_scan.rs} | 22 +-- .../src/task/barrier_manager/managed_state.rs | 16 +- .../src/task/barrier_manager/progress.rs | 45 +++-- src/stream/src/task/stream_manager.rs | 2 +- .../integration_tests/recovery/backfill.rs | 9 +- .../scale/cascade_materialized_view.rs | 8 +- .../integration_tests/scale/no_shuffle.rs | 2 +- .../tests/integration_tests/scale/plan.rs | 9 +- 43 files changed, 537 insertions(+), 560 deletions(-) rename src/stream/src/from_proto/{chain.rs => stream_scan.rs} (92%) diff --git a/dashboard/lib/streamPlan/parser.js b/dashboard/lib/streamPlan/parser.js index 4b5d7e79b50ba..139c8af1fa62b 100644 --- a/dashboard/lib/streamPlan/parser.js +++ b/dashboard/lib/streamPlan/parser.js @@ -77,7 +77,7 @@ class StreamNode extends Node { "hopWindow", "merge", "exchange", - "chain", + "streamScan", "batchPlan", "lookup", "arrange", @@ -393,7 +393,7 @@ export default class StreamPlanParser { } } - if (newNode.type === "chain" && newNode.typeInfo.upstreamActorIds) { + if (newNode.type === "streamScan" && newNode.typeInfo.upstreamActorIds) { for (let upStreamActorId of newNode.typeInfo.upstreamActorIds) { if (!this.actorId2Proto.has(upStreamActorId)) { continue diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index bebf85898a265..86b1b89d079ec 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -447,45 +447,48 @@ message ExchangeNode { DispatchStrategy strategy = 1; } -enum ChainType { +// Decides which kind of Executor will be used +enum StreamScanType { CHAIN_UNSPECIFIED = 0; - // CHAIN is corresponding to the chain executor. + // ChainExecutor CHAIN = 1; - // REARRANGE is corresponding to the rearranged chain executor. + // RearrangedChainExecutor REARRANGE = 2; - // BACKFILL is corresponding to the backfill executor. + // BackfillExecutor BACKFILL = 3; - // UPSTREAM_ONLY is corresponding to the chain executor, but doesn't consume the snapshot. + // ChainExecutor with upstream_only = true UPSTREAM_ONLY = 4; + // CdcBackfillExecutor CDC_BACKFILL = 5; } -// ChainNode is used for mv on mv. -// ChainNode is like a "UNION" on mv snapshot and streaming. So it takes two inputs with fixed order: -// 1. MergeNode (as a placeholder) for streaming read. -// 2. BatchPlanNode for snapshot read. -message ChainNode { +// StreamScanNode reads data from upstream table first, and then pass all events to downstream. +// It always these 2 inputs in the following order: +// 1. A MergeNode (as a placeholder) of upstream. +// 2. A BatchPlanNode for the snapshot read. +message StreamScanNode { uint32 table_id = 1; - // The columns from the upstream table that'll be internally required by this chain node. - // - For non-backfill chain node, it's the same as the output columns. - // - For backfill chain node, there're additionally primary key columns. + // The columns from the upstream table that'll be internally required by this stream scan node. + // - For non-backfill stream scan node, it's the same as the output columns. + // - For backfill stream scan node, there're additionally primary key columns. repeated int32 upstream_column_ids = 2; - // The columns to be output by this chain node. The index is based on the internal required columns. - // - For non-backfill chain node, it's simply all the columns. - // - For backfill chain node, this strips the primary key columns if they're unnecessary. + + // The columns to be output by this stream scan node. The index is based on the internal required columns. + // - For non-backfill stream scan node, it's simply all the columns. + // - For backfill stream scan node, this strips the primary key columns if they're unnecessary. repeated uint32 output_indices = 3; // Generally, the barrier needs to be rearranged during the MV creation process, so that data can // be flushed to shared buffer periodically, instead of making the first epoch from batch query extra - // large. However, in some cases, e.g., shared state, the barrier cannot be rearranged in ChainNode. - // ChainType is used to decide which implementation for the ChainNode. - ChainType chain_type = 4; + // large. However, in some cases, e.g., shared state, the barrier cannot be rearranged in StreamScanNode. + // StreamScanType is used to decide which implementation for the StreamScanNode. + StreamScanType stream_scan_type = 4; /// The state table used by Backfill operator for persisting internal state catalog.Table state_table = 5; @@ -493,7 +496,7 @@ message ChainNode { // The upstream materialized view info used by backfill. plan_common.StorageTableDesc table_desc = 7; - // The rate limit for the chain node. + // The rate limit for the stream scan node. optional uint32 rate_limit = 8; // Snapshot read every N barriers @@ -670,7 +673,7 @@ message StreamNode { HopWindowNode hop_window = 110; MergeNode merge = 111; ExchangeNode exchange = 112; - ChainNode chain = 113; + StreamScanNode stream_scan = 113; BatchPlanNode batch_plan = 114; LookupNode lookup = 115; ArrangeNode arrange = 116; @@ -791,7 +794,7 @@ enum FragmentTypeFlag { FRAGMENT_TYPE_FLAG_MVIEW = 2; FRAGMENT_TYPE_FLAG_SINK = 4; FRAGMENT_TYPE_FLAG_NOW = 8; // TODO: Remove this and insert a `BarrierRecv` instead. - FRAGMENT_TYPE_FLAG_CHAIN_NODE = 16; + FRAGMENT_TYPE_FLAG_STREAM_SCAN = 16; FRAGMENT_TYPE_FLAG_BARRIER_RECV = 32; FRAGMENT_TYPE_FLAG_VALUES = 64; FRAGMENT_TYPE_FLAG_DML = 128; @@ -817,7 +820,7 @@ message StreamFragmentGraph { bool requires_singleton = 4; // Number of table ids (stateful states) for this fragment. uint32 table_ids_cnt = 5; - // Mark the upstream table ids of this fragment, Used for fragments with `Chain`s. + // Mark the upstream table ids of this fragment, Used for fragments with `StreamScan`s. repeated uint32 upstream_table_ids = 6; } diff --git a/proto/stream_service.proto b/proto/stream_service.proto index dd964a6fa5908..d7b3edd3fe7d4 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -72,7 +72,7 @@ message BarrierCompleteRequest { } message BarrierCompleteResponse { message CreateMviewProgress { - uint32 chain_actor_id = 1; + uint32 backfill_actor_id = 1; bool done = 2; uint64 consumed_epoch = 3; uint64 consumed_rows = 4; diff --git a/src/common/src/catalog/external_table.rs b/src/common/src/catalog/external_table.rs index 8c4ed3f6d1bd5..28d43a7e97a0a 100644 --- a/src/common/src/catalog/external_table.rs +++ b/src/common/src/catalog/external_table.rs @@ -39,7 +39,7 @@ pub struct CdcTableDesc { pub value_indices: Vec, - /// properties will be passed into the ChainNode + /// properties will be passed into the StreamScanNode pub connect_properties: BTreeMap, } diff --git a/src/common/src/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index a6aa2082ad141..baa1dd4e1fb28 100644 --- a/src/common/src/util/stream_graph_visitor.rs +++ b/src/common/src/util/stream_graph_visitor.rs @@ -194,9 +194,9 @@ fn visit_stream_node_tables_inner( always!(node.state_table, "Sort"); } - // Chain - NodeBody::Chain(node) => { - optional!(node.state_table, "Chain") + // Stream Scan + NodeBody::StreamScan(node) => { + optional!(node.state_table, "StreamScan") } // Note: add internal tables for new nodes here. diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 463017e61a3e3..f1f18c43b2a70 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1625,7 +1625,7 @@ └── StreamExchange Single from 1 Fragment 1 - Chain { table: t, columns: [t.x, t.y, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + StreamTableScan { table: t, columns: [t.x, t.y, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1674,7 +1674,7 @@ └── StreamExchange Single from 1 Fragment 1 - Chain { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 3 ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml index e7196f7cf4fea..7eed762be53bc 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -42,7 +42,7 @@ Fragment 1 StreamProject { exprs: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id] } └── StreamFilter { predicate: (order_line.ol_delivery_d > '2007-01-02 00:00:00':Timestamp) } - └── Chain { table: order_line, columns: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -212,7 +212,7 @@ Fragment 5 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 18 } + └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 18 } ├── Upstream └── BatchPlanNode @@ -222,31 +222,31 @@ └── StreamExchange Hash([0]) from 8 Fragment 7 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 23 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 23 } ├── Upstream └── BatchPlanNode Fragment 8 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 24 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 24 } ├── Upstream └── BatchPlanNode Fragment 9 StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } └── StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } - └── Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 25 } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 25 } ├── Upstream └── BatchPlanNode Fragment 10 StreamProject { exprs: [item.i_id, item.i_name] } └── StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } - └── Chain { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 30 } + └── StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 30 } ├── Upstream └── BatchPlanNode Fragment 11 - Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 31 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 31 } ├── Upstream └── BatchPlanNode @@ -258,7 +258,7 @@ Fragment 13 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 36 } + └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 36 } ├── Upstream └── BatchPlanNode @@ -268,12 +268,12 @@ └── StreamExchange Hash([0]) from 16 Fragment 15 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 41 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 41 } ├── Upstream └── BatchPlanNode Fragment 16 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 42 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 42 } ├── Upstream └── BatchPlanNode @@ -463,25 +463,25 @@ Fragment 3 StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } └── StreamFilter { predicate: (customer.c_w_id = customer.c_w_id) AND (customer.c_d_id = customer.c_d_id) AND Like(customer.c_state, 'a%':Varchar) } - └── Chain { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 9 } + └── StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 9 } ├── Upstream └── BatchPlanNode Fragment 4 StreamFilter { predicate: (new_order.no_w_id = new_order.no_w_id) AND (new_order.no_d_id = new_order.no_d_id) AND (new_order.no_o_id = new_order.no_o_id) } - └── Chain { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], pk: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], dist: UpstreamHashShard(new_order.no_o_id, new_order.no_d_id, new_order.no_w_id) } { state table: 10 } + └── StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], pk: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], dist: UpstreamHashShard(new_order.no_o_id, new_order.no_d_id, new_order.no_w_id) } { state table: 10 } ├── Upstream └── BatchPlanNode Fragment 5 StreamFilter { predicate: (orders.o_d_id = orders.o_d_id) AND (orders.o_w_id = orders.o_w_id) AND (orders.o_entry_d > '2007-01-02 00:00:00':Timestamp) } - └── Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 15 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 15 } ├── Upstream └── BatchPlanNode Fragment 6 StreamFilter { predicate: (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_w_id = order_line.ol_w_id) } - └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 16 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 16 } ├── Upstream └── BatchPlanNode @@ -585,13 +585,13 @@ Fragment 2 StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d < '2032-01-02 00:00:00':Timestamp) } - └── Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 5 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -746,14 +746,14 @@ └── StreamExchange Hash([1, 2, 3]) from 5 Fragment 4 - Chain { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 13 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 13 } ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └── StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - └── Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 14 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 14 } ├── Upstream └── BatchPlanNode @@ -764,12 +764,12 @@ Fragment 7 StreamFilter { predicate: (order_line.ol_d_id = order_line.ol_d_id) } - └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 19 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 19 } ├── Upstream └── BatchPlanNode Fragment 8 - Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 20 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 20 } ├── Upstream └── BatchPlanNode @@ -781,7 +781,7 @@ Fragment 10 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'EUROPE':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 25 } + └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 25 } ├── Upstream └── BatchPlanNode @@ -791,12 +791,12 @@ └── StreamExchange Hash([0]) from 13 Fragment 12 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 30 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 30 } ├── Upstream └── BatchPlanNode Fragment 13 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 31 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 31 } ├── Upstream └── BatchPlanNode @@ -910,7 +910,7 @@ StreamStatelessSimpleAgg { aggs: [sum(order_line.ol_amount)] } └── StreamProject { exprs: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d >= '1997-01-01 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) AND (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 100000:Int32) } - └── Chain { table: order_line, columns: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1067,14 +1067,14 @@ └── StreamExchange Hash([3, 4]) from 6 Fragment 5 - Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 17 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 17 } ├── Upstream └── BatchPlanNode Fragment 6 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d <= '2032-01-02 00:00:00':Timestamp) } - └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 18 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 18 } ├── Upstream └── BatchPlanNode @@ -1084,17 +1084,17 @@ └── StreamExchange Hash([1, 2, 0]) from 9 Fragment 8 - Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 23 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 23 } ├── Upstream └── BatchPlanNode Fragment 9 - Chain { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 24 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 24 } ├── Upstream └── BatchPlanNode Fragment 10 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 25 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 25 } ├── Upstream └── BatchPlanNode @@ -1104,12 +1104,12 @@ └── StreamExchange Hash([0]) from 13 Fragment 12 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 30 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 30 } ├── Upstream └── BatchPlanNode Fragment 13 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 31 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 31 } ├── Upstream └── BatchPlanNode @@ -1355,12 +1355,12 @@ Fragment 5 StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d <= '2032-01-02 00:00:00':Timestamp) } - └── Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 17 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 17 } ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 18 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 18 } ├── Upstream └── BatchPlanNode @@ -1376,19 +1376,19 @@ Fragment 9 StreamFilter { predicate: (item.i_id < 1000:Int32) } - └── Chain { table: item, columns: [item.i_id], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 27 } + └── StreamTableScan { table: item, columns: [item.i_id], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 27 } ├── Upstream └── BatchPlanNode Fragment 10 StreamFilter { predicate: (stock.s_i_id < 1000:Int32) } - └── Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 28 } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 28 } ├── Upstream └── BatchPlanNode Fragment 11 StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_i_id = order_line.ol_i_id) AND (order_line.ol_i_id < 1000:Int32) } - └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 29 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 29 } ├── Upstream └── BatchPlanNode @@ -1398,12 +1398,12 @@ └── StreamExchange Hash([0]) from 14 Fragment 13 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 34 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 34 } ├── Upstream └── BatchPlanNode Fragment 14 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 35 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 35 } ├── Upstream └── BatchPlanNode @@ -1413,14 +1413,14 @@ └── StreamExchange Hash([0]) from 17 Fragment 16 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 40 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 40 } ├── Upstream └── BatchPlanNode Fragment 17 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'ASIA':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 41 } + └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 41 } ├── Upstream └── BatchPlanNode @@ -1642,12 +1642,12 @@ Fragment 4 StreamProject { exprs: [item.i_id] } └── StreamFilter { predicate: Like(item.i_data, '%BB':Varchar) } - └── Chain { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 13 } + └── StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 13 } ├── Upstream └── BatchPlanNode Fragment 5 - Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 14 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 14 } ├── Upstream └── BatchPlanNode @@ -1661,13 +1661,13 @@ └── StreamExchange Hash([0, 1, 2]) from 8 Fragment 7 - Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 19 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 19 } ├── Upstream └── BatchPlanNode Fragment 8 StreamFilter { predicate: (order_line.ol_i_id = order_line.ol_i_id) } - └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 20 ├── Upstream └── BatchPlanNode @@ -1678,12 +1678,12 @@ └── StreamExchange Hash([0]) from 11 Fragment 10 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 25 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 25 } ├── Upstream └── BatchPlanNode Fragment 11 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 26 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 26 } ├── Upstream └── BatchPlanNode @@ -1849,7 +1849,7 @@ Fragment 3 StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) } - └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 9 ├── Upstream └── BatchPlanNode @@ -1864,29 +1864,24 @@ └── StreamExchange Hash([1, 2, 3]) from 6 Fragment 5 - Chain { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 14 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 14 } ├── Upstream └── BatchPlanNode Fragment 6 StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - └── Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 15 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 15 } ├── Upstream └── BatchPlanNode Fragment 7 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 16 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 16 } ├── Upstream └── BatchPlanNode Table 0 { columns: [ customer_c_id, customer_c_last, customer_c_city, customer_c_phone, nation_n_name, sum(order_line_ol_amount), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } - Table 1 - ├── columns: [ order_line_ol_amount, customer_c_id, customer_c_last, customer_c_city, customer_c_phone, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ] - ├── primary key: [ $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $1 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] - ├── distribution key: [ 5 ] - └── read pk prefix len hint: 1 + Table 1 { columns: [ order_line_ol_amount, customer_c_id, customer_c_last, customer_c_city, customer_c_phone, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ], primary key: [ $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 5 ], read pk prefix len hint: 1 } Table 2 { columns: [ $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, customer_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2072,7 +2067,7 @@ └── StreamExchange Hash([0]) from 5 Fragment 4 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── state table: 11 ├── Upstream └── BatchPlanNode @@ -2080,13 +2075,14 @@ Fragment 5 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 12 } + └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + ├── state table: 12 ├── Upstream └── BatchPlanNode Fragment 6 StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - └── Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } ├── state table: 13 ├── Upstream └── BatchPlanNode @@ -2139,12 +2135,7 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 7 - ├── columns: [ supplier_s_suppkey, supplier_s_nationkey ] - ├── primary key: [ $1 ASC, $0 ASC ] - ├── value indices: [ 0, 1 ] - ├── distribution key: [ 1 ] - └── read pk prefix len hint: 1 + Table 7 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } Table 8 ├── columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ] @@ -2247,13 +2238,13 @@ └── StreamExchange Hash([0, 1, 2]) from 3 Fragment 2 - Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 5 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 StreamFilter { predicate: (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) } - └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -2350,14 +2341,14 @@ └── StreamExchange Hash([1, 2, 3]) from 4 Fragment 3 - Chain { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 6 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 6 } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └── StreamFilter { predicate: (orders.o_carrier_id > 8:Int32) } - └── Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } ├── state table: 7 ├── Upstream └── BatchPlanNode @@ -2449,13 +2440,13 @@ Fragment 2 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-02 00:00:00':Timestamp) } - └── Chain { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 5 ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 6 } + StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -2541,13 +2532,13 @@ └── StreamExchange Hash([1]) from 4 Fragment 3 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 8 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 8 } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] } - └── Chain { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } { state table: 9 } + └── StreamTableScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } { state table: 9 } ├── Upstream └── BatchPlanNode @@ -2559,7 +2550,7 @@ Fragment 6 StreamHashAgg { group_key: [$expr2], aggs: [max(revenue1.total_revenue), count] } { intermediate state table: 13, state tables: [ 12 ], distinct tables: [] } └── StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no, Vnode(revenue1.supplier_no) as $expr2] } - └── Chain { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } { state table: 14 } + └── StreamTableScan { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } { state table: 14 } ├── Upstream └── BatchPlanNode @@ -2687,20 +2678,20 @@ └── StreamExchange Hash([0]) from 4 Fragment 3 - Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 10 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 10 } ├── Upstream └── BatchPlanNode Fragment 4 StreamFilter { predicate: Not(Like(item.i_data, 'zz%':Varchar)) } - └── Chain { table: item, columns: [item.i_id, item.i_name, item.i_price, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 11 } + └── StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_price, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 11 } ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [supplier.s_suppkey] } └── StreamFilter { predicate: Like(supplier.s_comment, '%bad%':Varchar) } - └── Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 12 } + └── StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 12 } ├── Upstream └── BatchPlanNode @@ -2867,7 +2858,7 @@ Fragment 2 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_quantity::Decimal as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - └── Chain { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 5 ├── Upstream └── BatchPlanNode @@ -2875,12 +2866,12 @@ Fragment 3 StreamProject { exprs: [item.i_id] } └── StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } - └── Chain { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 11 } + └── StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 11 } ├── Upstream └── BatchPlanNode Fragment 4 - Chain { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 12 } + StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 12 } ├── Upstream └── BatchPlanNode @@ -2990,17 +2981,17 @@ └── StreamExchange Hash([1, 2, 3]) from 3 Fragment 2 - Chain { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 9 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 9 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 10 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 10 } ├── Upstream └── BatchPlanNode Fragment 4 - Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 11 } + StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 11 } ├── Upstream └── BatchPlanNode @@ -3095,7 +3086,7 @@ Fragment 2 StreamProject { exprs: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 10:Int32) } - └── Chain { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 5 ├── Upstream └── BatchPlanNode @@ -3103,7 +3094,7 @@ Fragment 3 StreamProject { exprs: [item.i_id, item.i_data] } └── StreamFilter { predicate: (item.i_price >= 1:Decimal) AND (item.i_price <= 400000:Decimal) } - └── Chain { table: item, columns: [item.i_id, item.i_data, item.i_price], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 6 } + └── StreamTableScan { table: item, columns: [item.i_id, item.i_data, item.i_price], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -3226,14 +3217,14 @@ └── StreamExchange Hash([0]) from 4 Fragment 3 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 8 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 8 } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 9 } + └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 9 } ├── Upstream └── BatchPlanNode @@ -3253,14 +3244,14 @@ └── StreamExchange Hash([0]) from 8 Fragment 6 - Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 19 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 19 } ├── Upstream └── BatchPlanNode Fragment 7 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } - └── Chain { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 20 ├── Upstream └── BatchPlanNode @@ -3268,7 +3259,7 @@ Fragment 8 StreamProject { exprs: [item.i_id] } └── StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) } - └── Chain { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 21 } + └── StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 21 } ├── Upstream └── BatchPlanNode @@ -3439,7 +3430,7 @@ Fragment 4 StreamFilter { predicate: (stock.s_w_id = stock.s_w_id) } - └── Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 13 } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 13 } ├── Upstream └── BatchPlanNode @@ -3451,12 +3442,12 @@ └── StreamExchange Hash([0, 1, 2]) from 7 Fragment 6 - Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 18 } + StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 18 } ├── Upstream └── BatchPlanNode Fragment 7 - Chain { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 19 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 19 } ├── Upstream └── BatchPlanNode @@ -3466,20 +3457,20 @@ └── StreamExchange Hash([0]) from 10 Fragment 9 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 24 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 24 } ├── Upstream └── BatchPlanNode Fragment 10 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 25 } + └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 25 } ├── Upstream └── BatchPlanNode Fragment 11 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 26 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 26 } ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml index 818fd88b30a20..97d77873ff90b 100644 --- a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml @@ -34,12 +34,12 @@ └── StreamExchange Hash([2, 4, 3]) from 5 Fragment 2 - Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 0 } + StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } + StreamTableScan { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -86,12 +86,12 @@ └── StreamExchange Hash([2, 4, 3]) from 5 Fragment 2 - Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 0 } + StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } + StreamTableScan { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -138,12 +138,12 @@ └── StreamExchange Hash([2, 4, 3]) from 5 Fragment 2 - Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 0 } + StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } + StreamTableScan { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -190,12 +190,12 @@ └── StreamExchange Hash([2, 4, 3]) from 5 Fragment 2 - Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 0 } + StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } + StreamTableScan { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -246,7 +246,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: a, columns: [a.k1, a.v, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + StreamTableScan { table: a, columns: [a.k1, a.v, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -306,7 +306,7 @@ ├── intermediate state table: 1 ├── state tables: [ 0 ] ├── distinct tables: [] - └── Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } + └── StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -362,11 +362,8 @@ StreamMaterialize { columns: [max_v, ak1k2.k1(hidden)], stream_key: [ak1k2.k1], pk_columns: [ak1k2.k1], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamProject { exprs: [max(ak1k2.v), ak1k2.k1] } - └── StreamHashAgg { group_key: [ak1k2.k1], aggs: [max(ak1k2.v), count] } - ├── intermediate state table: 1 - ├── state tables: [ 0 ] - ├── distinct tables: [] - └── Chain { table: ak1k2, columns: [ak1k2.k1, ak1k2.v, ak1k2.k2, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1) } + └── StreamHashAgg { group_key: [ak1k2.k1], aggs: [max(ak1k2.v), count] } { intermediate state table: 1, state tables: [ 0 ], distinct tables: [] } + └── StreamTableScan { table: ak1k2, columns: [ak1k2.k1, ak1k2.v, ak1k2.k2, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -378,12 +375,7 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ ak1k2_k1, max(ak1k2_v), count ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 + Table 1 { columns: [ ak1k2_k1, max(ak1k2_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 2 ├── columns: [ vnode, k1, k2, a__row_id, ak1k2_backfill_finished, ak1k2_row_count ] @@ -393,12 +385,7 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 4294967294 - ├── columns: [ max_v, ak1k2.k1 ] - ├── primary key: [ $1 ASC ] - ├── value indices: [ 0, 1 ] - ├── distribution key: [ 1 ] - └── read pk prefix len hint: 1 + Table 4294967294 { columns: [ max_v, ak1k2.k1 ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: aggk2_from_Ak1k2 before: @@ -431,7 +418,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: ak1k2, columns: [ak1k2.k2, ak1k2.v, ak1k2.k1, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1) } + StreamTableScan { table: ak1k2, columns: [ak1k2.k2, ak1k2.v, ak1k2.k1, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -488,7 +475,7 @@ ├── materialized table: 4294967294 └── StreamProject { exprs: [sum(ak1k2.v), ak1k2.k1, ak1k2.k2] } └── StreamHashAgg { group_key: [ak1k2.k1, ak1k2.k2], aggs: [sum(ak1k2.v), count] } { intermediate state table: 0, state tables: [], distinct tables: [] } - └── Chain { table: ak1k2, columns: [ak1k2.k1, ak1k2.k2, ak1k2.v, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1) } + └── StreamTableScan { table: ak1k2, columns: [ak1k2.k1, ak1k2.k2, ak1k2.v, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -538,7 +525,7 @@ ├── materialized table: 4294967294 └── StreamProject { exprs: [sum(ak1.v), ak1.k1, ak1.k2] } └── StreamHashAgg { group_key: [ak1.k1, ak1.k2], aggs: [sum(ak1.v), count] } { intermediate state table: 0, state tables: [], distinct tables: [] } - └── Chain { table: ak1, columns: [ak1.k1, ak1.k2, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } + └── StreamTableScan { table: ak1, columns: [ak1.k1, ak1.k2, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -608,7 +595,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } ├── state table: 3 ├── Upstream └── BatchPlanNode @@ -710,7 +697,7 @@ └── StreamExchange Hash([0, 1]) from 2 Fragment 2 - Chain { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } ├── state table: 3 ├── Upstream └── BatchPlanNode @@ -812,7 +799,7 @@ └── StreamExchange Hash([0, 1]) from 2 Fragment 2 - Chain { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } ├── state table: 3 ├── Upstream └── BatchPlanNode @@ -893,7 +880,7 @@ └── StreamExchange Hash([0, 1]) from 1 Fragment 1 - Chain { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 3 } + StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 3 } ├── Upstream └── BatchPlanNode @@ -987,12 +974,12 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 4 } + StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 4 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 6 } + StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -1082,12 +1069,12 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 } + StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 6 } + StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -1190,12 +1177,12 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 } + StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } { state table: 7 } + StreamTableScan { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } { state table: 7 } ├── Upstream └── BatchPlanNode @@ -1291,7 +1278,7 @@ ├── materialized table: 4294967294 └── StreamHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: [t1.row_id, t1.uid, t1.v, t1.created_at, window_start, window_end, t1._row_id] } └── StreamFilter { predicate: IsNotNull(t1.created_at) } - └── Chain { table: t1, columns: [t1.row_id, t1.uid, t1.v, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 0 } + └── StreamTableScan { table: t1, columns: [t1.row_id, t1.uid, t1.v, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml b/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml index e7c6f8d0feb00..cd3019382bd66 100644 --- a/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml +++ b/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml @@ -116,7 +116,7 @@ Fragment 1 StreamProject { exprs: [TumbleStart(t.a, '01:00:00':Interval) as $expr1, t.b, t._row_id], output_watermarks: [$expr1] } - └── Chain { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 1 } + └── StreamTableScan { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 1 } ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/except.yaml b/src/frontend/planner_test/tests/testdata/output/except.yaml index 1e27a7b74c0f0..a98c834a19980 100644 --- a/src/frontend/planner_test/tests/testdata/output/except.yaml +++ b/src/frontend/planner_test/tests/testdata/output/except.yaml @@ -46,12 +46,12 @@ └── StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 5 } + StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 6 } + StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -149,12 +149,12 @@ └── StreamExchange Hash([0, 1, 2]) from 3 Fragment 2 - Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 } + StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 } + StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 } ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/intersect.yaml b/src/frontend/planner_test/tests/testdata/output/intersect.yaml index 91839346824ec..5badfae8c9f73 100644 --- a/src/frontend/planner_test/tests/testdata/output/intersect.yaml +++ b/src/frontend/planner_test/tests/testdata/output/intersect.yaml @@ -46,12 +46,12 @@ └── StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 5 } + StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 6 } + StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -149,12 +149,12 @@ └── StreamExchange Hash([0, 1, 2]) from 3 Fragment 2 - Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 } + StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 } + StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 } ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/join.yaml b/src/frontend/planner_test/tests/testdata/output/join.yaml index a61d2a0d73327..387862e92861e 100644 --- a/src/frontend/planner_test/tests/testdata/output/join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join.yaml @@ -686,17 +686,17 @@ └── StreamExchange Hash([0]) from 4 Fragment 3 - Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 8 } + StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 8 } ├── Upstream └── BatchPlanNode Fragment 4 - Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 9 } + StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 9 } ├── Upstream └── BatchPlanNode Fragment 5 - Chain { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 10 } + StreamTableScan { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 10 } ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index 1399ee708ee9c..cd3fb179a43f7 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -59,7 +59,7 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── state table: 0 ├── Upstream └── BatchPlanNode @@ -107,7 +107,7 @@ StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price::Decimal) as $expr1, bid.date_time, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── state table: 0 ├── Upstream └── BatchPlanNode @@ -148,7 +148,7 @@ Fragment 0 StreamMaterialize { columns: [auction, price, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: NoCheck } { materialized table: 4294967294 } └── StreamFilter { predicate: (((((bid.auction = 1007:Int32) OR (bid.auction = 1020:Int32)) OR (bid.auction = 2001:Int32)) OR (bid.auction = 2019:Int32)) OR (bid.auction = 2087:Int32)) } - └── Chain { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode @@ -208,13 +208,13 @@ Fragment 2 StreamProject { exprs: [auction.id, auction.seller] } └── StreamFilter { predicate: (auction.category = 10:Int32) } - └── Chain { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 } + └── StreamTableScan { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 } ├── Upstream └── BatchPlanNode Fragment 3 StreamFilter { predicate: (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)) } - └── Chain { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 } + └── StreamTableScan { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 } ├── Upstream └── BatchPlanNode @@ -316,13 +316,13 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: auction, columns: [auction.id, auction.date_time, auction.expires, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + StreamTableScan { table: auction, columns: [auction.id, auction.date_time, auction.expires, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── state table: 7 ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── state table: 8 ├── Upstream └── BatchPlanNode @@ -364,12 +364,7 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 6 - ├── columns: [ bid_auction, bid__row_id, _degree ] - ├── primary key: [ $0 ASC, $1 ASC ] - ├── value indices: [ 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 + Table 6 { columns: [ bid_auction, bid__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 ├── columns: [ vnode, id, auction_backfill_finished, auction_row_count ] @@ -499,7 +494,7 @@ Fragment 3 StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } └── StreamFilter { predicate: IsNotNull(bid.date_time) } - └── Chain { table: bid, columns: [bid.auction, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 5 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 5 } ├── Upstream └── BatchPlanNode @@ -587,12 +582,12 @@ └── StreamExchange Hash([0]) from 4 Fragment 3 - Chain { table: auction, columns: [auction.id, auction.date_time, auction.expires, auction.seller], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 6 } + StreamTableScan { table: auction, columns: [auction.id, auction.date_time, auction.expires, auction.seller], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 6 } ├── Upstream └── BatchPlanNode Fragment 4 - Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 7 } + StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 7 } ├── Upstream └── BatchPlanNode @@ -679,12 +674,12 @@ └── StreamExchange Hash([0]) from 4 Fragment 3 - Chain { table: auction, columns: [auction.id, auction.date_time, auction.expires, auction.seller], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 6 } + StreamTableScan { table: auction, columns: [auction.id, auction.date_time, auction.expires, auction.seller], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 6 } ├── Upstream └── BatchPlanNode Fragment 4 - Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 7 } + StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 7 } ├── Upstream └── BatchPlanNode @@ -764,7 +759,7 @@ └── StreamExchange Hash([1]) from 2 Fragment 1 - Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 4 } + StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 4 } ├── Upstream └── BatchPlanNode @@ -775,7 +770,7 @@ Fragment 3 StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, bid.price, bid._row_id] } - └── Chain { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } + └── StreamTableScan { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -887,14 +882,14 @@ └── StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] } { intermediate state table: 4, state tables: [], distinct tables: [] } └── StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } └── StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } - └── Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 } + └── StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] } └── StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] } - └── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } + └── StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } ├── Upstream └── BatchPlanNode @@ -978,13 +973,13 @@ └── StreamExchange Hash([0]) from 2 Fragment 1 - Chain { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + StreamTableScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── state table: 5 ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } + StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -1042,7 +1037,7 @@ StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr2, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode @@ -1112,7 +1107,7 @@ Fragment 1 StreamProject { exprs: [bid.bidder, $expr1, AddWithTimeZone($expr1, '00:00:10':Interval, 'UTC':Varchar) as $expr2, bid._row_id], output_watermarks: [$expr1, $expr2] } └── StreamProject { exprs: [bid.bidder, bid.p_time, TumbleStart(bid.p_time, '00:00:10':Interval) as $expr1, bid._row_id], output_watermarks: [bid.p_time, $expr1] } - └── Chain { table: bid, columns: [bid.bidder, bid.p_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 1 } + └── StreamTableScan { table: bid, columns: [bid.bidder, bid.p_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -1161,12 +1156,12 @@ Fragment 2 StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } { state table: 1 } + StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -1228,7 +1223,7 @@ StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: NoCheck } { materialized table: 4294967294 } └── StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, bid.date_time, bid.extra, bid._row_id] } └── StreamFilter { predicate: ((0.908:Decimal * bid.price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * bid.price::Decimal) < 50000000:Decimal) } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode @@ -1282,7 +1277,7 @@ Fragment 1 StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, bid.price, bid.bidder, bid.auction, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 3 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 3 } ├── Upstream └── BatchPlanNode @@ -1348,7 +1343,7 @@ Fragment 2 StreamExpand { column_subsets: [[$expr1], [$expr1, bid.bidder], [$expr1, bid.auction]] } └── StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, bid.price, bid.bidder, bid.auction, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 2 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 2 } ├── Upstream └── BatchPlanNode @@ -1415,7 +1410,7 @@ Fragment 1 StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr2, bid.price, bid.bidder, bid.auction, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 3 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 3 } ├── Upstream └── BatchPlanNode @@ -1483,7 +1478,7 @@ Fragment 2 StreamExpand { column_subsets: [[bid.channel, $expr1, $expr2], [bid.channel, $expr1, bid.bidder], [bid.channel, $expr1, bid.auction]] } └── StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr2, bid.price, bid.bidder, bid.auction, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 3 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 3 } ├── Upstream └── BatchPlanNode @@ -1547,7 +1542,7 @@ Fragment 1 StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr1, bid.price, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 1 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -1595,7 +1590,7 @@ └── StreamExchange Hash([1, 0]) from 1 Fragment 1 - Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1655,7 +1650,7 @@ └── StreamExchange Hash([1, 0]) from 1 Fragment 1 - Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 1 } + StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -1707,7 +1702,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 1 } + StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -1763,7 +1758,8 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 2 } + StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1830,13 +1826,13 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 4 } + StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 4 } ├── Upstream └── BatchPlanNode Fragment 3 StreamFilter { predicate: (auction.category = 10:Int32) } - └── Chain { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 5 } + └── StreamTableScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 5 } ├── Upstream └── BatchPlanNode @@ -1890,7 +1886,7 @@ StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: NoCheck } { materialized table: 4294967294 } └── StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, Case((Lower(bid.channel) = 'apple':Varchar), '0':Varchar, (Lower(bid.channel) = 'google':Varchar), '1':Varchar, (Lower(bid.channel) = 'facebook':Varchar), '2':Varchar, (Lower(bid.channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(bid.url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1, bid._row_id] } └── StreamFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(bid.url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(bid.channel), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode @@ -1924,7 +1920,7 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: NoCheck } { materialized table: 4294967294 } └── StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr1, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr2, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr3, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } + └── StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode @@ -1988,12 +1984,12 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 } + StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } + StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -2104,12 +2100,12 @@ └── StreamExchange Broadcast from 3 Fragment 1 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } + StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 } + StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 } ├── Upstream └── BatchPlanNode @@ -2124,7 +2120,7 @@ └── StreamExchange Hash([0]) from 5 Fragment 5 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 11 } + StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 11 } ├── Upstream └── BatchPlanNode @@ -2266,13 +2262,14 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── state table: 4 ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } + StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -2388,13 +2385,14 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── state table: 4 ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } + StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -2514,12 +2512,12 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } + StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 } + StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 } ├── Upstream └── BatchPlanNode @@ -2666,13 +2664,13 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: auction, columns: [auction.id, auction.date_time, auction.expires], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + StreamTableScan { table: auction, columns: [auction.id, auction.date_time, auction.expires], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── state table: 10 ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── state table: 11 ├── Upstream └── BatchPlanNode @@ -2684,12 +2682,7 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ min(min(max(bid_price))), count ] - ├── primary key: [] - ├── value indices: [ 0, 1 ] - ├── distribution key: [] - └── read pk prefix len hint: 0 + Table 1 { columns: [ min(min(max(bid_price))), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 2 ├── columns: [ $expr1, max(bid_price), auction_id ] @@ -2728,12 +2721,7 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 7 - ├── columns: [ auction_id, _degree ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 + Table 7 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 8 ├── columns: [ bid_auction, bid_price, bid_date_time, bid__row_id ] diff --git a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml index 54926222e976b..07fb3c75a0f2b 100644 --- a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml @@ -721,7 +721,8 @@ └── StreamExchange Hash([0, 1]) from 1 Fragment 1 - Chain { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 1 } + StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + ├── state table: 1 ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index 15404d6d863ab..d446b23fab883 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -261,7 +261,7 @@ └── StreamExchange Hash([0]) from 3 Fragment 3 - Chain { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 6 ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml index 264da7ee55b98..99f91e6eb8899 100644 --- a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml @@ -29,7 +29,7 @@ ├── intermediate state table: 1 ├── state tables: [ 0 ] ├── distinct tables: [] - └── Chain { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } + └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -86,7 +86,7 @@ ├── intermediate state table: 0 ├── state tables: [] ├── distinct tables: [] - └── Chain { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } + └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -136,7 +136,7 @@ ├── intermediate state table: 0 ├── state tables: [] ├── distinct tables: [] - └── Chain { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } + └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -189,7 +189,7 @@ ├── state tables: [ 0 ] ├── distinct tables: [] └── StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } - └── Chain { table: s, columns: [s.v, s.s, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } + └── StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -258,7 +258,7 @@ ├── state tables: [ 2 ] ├── distinct tables: [] └── StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } - └── Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └── StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 4 ├── Upstream └── BatchPlanNode @@ -333,7 +333,7 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [max(ao.v)] } - └── Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + └── StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -390,7 +390,7 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [sum(t.v)] } - └── Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └── StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -442,7 +442,7 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [sum(ao.v)] } - └── Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + └── StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -499,7 +499,7 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [count(t.v)] } - └── Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └── StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -551,7 +551,7 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [count(ao.v)] } - └── Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + └── StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -608,7 +608,7 @@ Fragment 1 StreamProject { exprs: [t.s, ',':Varchar, t.o, t._row_id] } - └── Chain { table: t, columns: [t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └── StreamTableScan { table: t, columns: [t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -667,7 +667,7 @@ Fragment 1 StreamProject { exprs: [ao.s, ',':Varchar, ao.o, ao._row_id] } - └── Chain { table: ao, columns: [ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + └── StreamTableScan { table: ao, columns: [ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -736,7 +736,7 @@ ├── state tables: [ 2 ] ├── distinct tables: [] └── StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } - └── Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └── StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 4 ├── Upstream └── BatchPlanNode @@ -811,7 +811,7 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [max(ao.v), count(ao.v)] } - └── Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + └── StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -868,7 +868,7 @@ Fragment 1 StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } - └── Chain { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └── StreamTableScan { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -927,7 +927,7 @@ Fragment 1 StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } - └── Chain { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + └── StreamTableScan { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -991,7 +991,7 @@ Fragment 1 StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } - └── Chain { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └── StreamTableScan { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 3 ├── Upstream └── BatchPlanNode @@ -1057,7 +1057,7 @@ Fragment 1 StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } - └── Chain { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + └── StreamTableScan { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1120,7 +1120,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + StreamTableScan { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1178,7 +1178,7 @@ ├── intermediate state table: 1 ├── state tables: [ 0 ] ├── distinct tables: [] - └── Chain { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } + └── StreamTableScan { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1241,7 +1241,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } + StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1298,7 +1298,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + StreamTableScan { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1354,7 +1354,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + StreamTableScan { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1405,7 +1405,7 @@ ├── intermediate state table: 0 ├── state tables: [] ├── distinct tables: [] - └── Chain { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } + └── StreamTableScan { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1461,7 +1461,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } + StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1511,7 +1511,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + StreamTableScan { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1567,7 +1567,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + StreamTableScan { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1618,7 +1618,7 @@ ├── intermediate state table: 0 ├── state tables: [] ├── distinct tables: [] - └── Chain { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } + └── StreamTableScan { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1674,7 +1674,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } + StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1724,7 +1724,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + StreamTableScan { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1783,7 +1783,7 @@ Fragment 1 StreamProject { exprs: [t.k, t.s, ',':Varchar, t.o, t._row_id] } - └── Chain { table: t, columns: [t.k, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └── StreamTableScan { table: t, columns: [t.k, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1844,7 +1844,7 @@ ├── state tables: [ 0 ] ├── distinct tables: [] └── StreamProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o, tk.t._row_id] } - └── Chain { table: tk, columns: [tk.k, tk.o, tk.s, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } + └── StreamTableScan { table: tk, columns: [tk.k, tk.o, tk.s, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1910,7 +1910,7 @@ Fragment 1 StreamProject { exprs: [s.k, s.s, ',':Varchar, s.o, s.t._row_id] } - └── Chain { table: s, columns: [s.k, s.o, s.s, s.t._row_id], pk: [s.t._row_id], dist: Single } + └── StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1969,7 +1969,7 @@ Fragment 1 StreamProject { exprs: [ao.k, ao.s, ',':Varchar, ao.o, ao._row_id] } - └── Chain { table: ao, columns: [ao.k, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } + └── StreamTableScan { table: ao, columns: [ao.k, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml index ac4e599532ca6..abfded30edb49 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml @@ -64,7 +64,8 @@ ├── StreamDynamicFilter { predicate: (t1.ts >= $expr1), output_watermarks: [t1.ts], output: [t1.ts, t1._row_id], cleaned_by_watermark: true } │ ├── left table: 2 │ ├── right table: 3 - │ ├── Chain { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 4 } + │ ├── StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ ├── state table: 4 │ │ ├── Upstream │ │ └── BatchPlanNode │ └── StreamExchange Broadcast from 1 diff --git a/src/frontend/planner_test/tests/testdata/output/tpch.yaml b/src/frontend/planner_test/tests/testdata/output/tpch.yaml index 52c4a4b813198..1037300ef9669 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch.yaml @@ -175,7 +175,7 @@ StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, $expr1, ($expr1 * (1:Decimal + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamProject { exprs: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } - └── Chain { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 3 } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 3 } ├── Upstream └── BatchPlanNode @@ -403,17 +403,17 @@ Fragment 4 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 14 } + └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 14 } ├── Upstream └── BatchPlanNode Fragment 5 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 15 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 15 } ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 16 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 16 } ├── Upstream └── BatchPlanNode @@ -433,7 +433,7 @@ └── StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } { intermediate state table: 27, state tables: [], distinct tables: [] } └── StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 28 } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 28 } ├── Upstream └── BatchPlanNode @@ -449,12 +449,12 @@ Fragment 11 StreamFilter { predicate: IsNotNull(partsupp.ps_partkey) } - └── Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 37 } + └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 37 } ├── Upstream └── BatchPlanNode Fragment 12 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 38 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 38 } ├── Upstream └── BatchPlanNode @@ -466,24 +466,24 @@ Fragment 14 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 43 } + └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 43 } ├── Upstream └── BatchPlanNode Fragment 15 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 44 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 44 } ├── Upstream └── BatchPlanNode Fragment 16 StreamProject { exprs: [part.p_partkey, part.p_mfgr] } └── StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 49 } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 49 } ├── Upstream └── BatchPlanNode Fragment 17 - Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 50 } + StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 50 } ├── Upstream └── BatchPlanNode @@ -717,33 +717,27 @@ Fragment 4 StreamProject { exprs: [customer.c_custkey] } └── StreamFilter { predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) } - └── Chain { table: customer, columns: [customer.c_custkey, customer.c_mktsegment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 11 } + └── StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_mktsegment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 11 } ├── Upstream └── BatchPlanNode Fragment 5 StreamFilter { predicate: (orders.o_orderdate < '1995-03-29':Date) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 12 } + └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 12 } ├── Upstream └── BatchPlanNode Fragment 6 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate > '1995-03-29':Date) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── state table: 13 ├── Upstream └── BatchPlanNode Table 0 { columns: [ lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2 ], primary key: [ $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 - ├── columns: [ lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2 ] - ├── primary key: [ $4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4 ] - ├── distribution key: [ 0, 2, 3 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 4 + Table 1 { columns: [ lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2 ], primary key: [ $4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 1, vnode column idx: 4 } Table 2 { columns: [ lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, sum($expr1), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } @@ -876,7 +870,7 @@ Fragment 3 StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } └── StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── state table: 7 ├── Upstream └── BatchPlanNode @@ -884,7 +878,7 @@ Fragment 4 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── state table: 8 ├── Upstream └── BatchPlanNode @@ -1067,13 +1061,13 @@ Fragment 4 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'MIDDLE EAST':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 11 } + └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 11 } ├── Upstream └── BatchPlanNode Fragment 5 StreamFilter { predicate: (nation.n_nationkey = nation.n_nationkey) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 12 } + └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 12 } ├── Upstream └── BatchPlanNode @@ -1094,12 +1088,12 @@ Fragment 8 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 21 } + └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 21 } ├── Upstream └── BatchPlanNode Fragment 9 - Chain { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 22 } + StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 22 } ├── Upstream └── BatchPlanNode @@ -1109,12 +1103,12 @@ └── StreamExchange Hash([0]) from 12 Fragment 11 - Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 27 } + StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 27 } ├── Upstream └── BatchPlanNode Fragment 12 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 28 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 28 } ├── Upstream └── BatchPlanNode @@ -1237,7 +1231,7 @@ StreamStatelessSimpleAgg { aggs: [sum($expr1)] } └── StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Decimal) } - └── Chain { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1418,18 +1412,18 @@ └── StreamExchange Hash([1]) from 6 Fragment 5 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 15 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 15 } ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 16 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 16 } ├── Upstream └── BatchPlanNode Fragment 7 StreamFilter { predicate: (lineitem.l_shipdate >= '1983-01-01':Date) AND (lineitem.l_shipdate <= '2000-12-31':Date) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── state table: 17 ├── Upstream └── BatchPlanNode @@ -1445,17 +1439,17 @@ └── StreamExchange Hash([1]) from 11 Fragment 10 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 26 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 26 } ├── Upstream └── BatchPlanNode Fragment 11 - Chain { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 27 } + StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 27 } ├── Upstream └── BatchPlanNode Fragment 12 - Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 28 } + StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 28 } ├── Upstream └── BatchPlanNode @@ -1723,17 +1717,17 @@ Fragment 5 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'ASIA':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 15 } + └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 15 } ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 16 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 16 } ├── Upstream └── BatchPlanNode Fragment 7 - Chain { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 17 } + StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 17 } ├── Upstream └── BatchPlanNode @@ -1757,12 +1751,12 @@ └── StreamExchange Hash([1]) from 12 Fragment 11 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 30 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 30 } ├── Upstream └── BatchPlanNode Fragment 12 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 31 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 31 } ├── Upstream └── BatchPlanNode @@ -1774,18 +1768,18 @@ Fragment 14 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: (part.p_type = 'PROMO ANODIZED STEEL':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 36 } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 36 } ├── Upstream └── BatchPlanNode Fragment 15 - Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 37 } + StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 37 } ├── Upstream └── BatchPlanNode Fragment 16 StreamFilter { predicate: (orders.o_orderdate >= '1995-01-01':Date) AND (orders.o_orderdate <= '1996-12-31':Date) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 38 } + └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 38 } ├── Upstream └── BatchPlanNode @@ -2037,13 +2031,13 @@ Fragment 4 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: Like(part.p_name, '%yellow%':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 11 } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 11 } ├── Upstream └── BatchPlanNode Fragment 5 StreamFilter { predicate: (partsupp.ps_suppkey = partsupp.ps_suppkey) } - └── Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 12 } + └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 12 } ├── Upstream └── BatchPlanNode @@ -2053,12 +2047,12 @@ └── StreamExchange Hash([1]) from 8 Fragment 7 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 21 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 21 } ├── Upstream └── BatchPlanNode Fragment 8 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 22 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 22 } ├── Upstream └── BatchPlanNode @@ -2068,13 +2062,13 @@ └── StreamExchange Hash([0]) from 11 Fragment 10 - Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 27 } + StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 27 } ├── Upstream └── BatchPlanNode Fragment 11 StreamFilter { predicate: (lineitem.l_partkey = lineitem.l_partkey) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 28 } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 28 } ├── Upstream └── BatchPlanNode @@ -2275,12 +2269,12 @@ └── StreamExchange Hash([3]) from 4 Fragment 3 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 11 } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 11 } ├── Upstream └── BatchPlanNode Fragment 4 - Chain { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 12 } + StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 12 } ├── Upstream └── BatchPlanNode @@ -2292,14 +2286,14 @@ Fragment 6 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_returnflag = 'R':Varchar) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_returnflag], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 17 } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_returnflag], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 17 } ├── Upstream └── BatchPlanNode Fragment 7 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 18 } + └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 18 } ├── Upstream └── BatchPlanNode @@ -2536,20 +2530,20 @@ └── StreamExchange Hash([0]) from 6 Fragment 5 - Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── state table: 13 ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 14 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 14 } ├── Upstream └── BatchPlanNode Fragment 7 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 15 } + └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 15 } ├── Upstream └── BatchPlanNode @@ -2708,14 +2702,14 @@ └── StreamExchange Hash([0]) from 4 Fragment 3 - Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 7 } + StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 7 } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } └── StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 8 } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 8 } ├── Upstream └── BatchPlanNode @@ -2847,7 +2841,7 @@ └── StreamExchange Hash([1]) from 4 Fragment 3 - Chain { table: customer, columns: [customer.c_custkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + StreamTableScan { table: customer, columns: [customer.c_custkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── state table: 8 ├── Upstream └── BatchPlanNode @@ -2855,7 +2849,7 @@ Fragment 4 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── state table: 9 ├── Upstream └── BatchPlanNode @@ -2888,12 +2882,7 @@ Table 5 { columns: [ customer_c_custkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 - ├── columns: [ orders_o_orderkey, orders_o_custkey ] - ├── primary key: [ $1 ASC, $0 ASC ] - ├── value indices: [ 0, 1 ] - ├── distribution key: [ 1 ] - └── read pk prefix len hint: 1 + Table 6 { columns: [ orders_o_orderkey, orders_o_custkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } Table 7 ├── columns: [ orders_o_custkey, orders_o_orderkey, _degree ] @@ -3002,13 +2991,13 @@ Fragment 2 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } - └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── state table: 5 ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 6 } + StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -3159,7 +3148,7 @@ └── StreamExchange NoShuffle from 4 Fragment 3 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── state table: 10 ├── Upstream └── BatchPlanNode @@ -3172,7 +3161,7 @@ Fragment 5 StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } - └── Chain { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── state table: 12 ├── Upstream └── BatchPlanNode @@ -3351,21 +3340,21 @@ └── StreamExchange Hash([0]) from 5 Fragment 4 - Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── state table: 12 ├── Upstream └── BatchPlanNode Fragment 5 StreamFilter { predicate: (part.p_brand <> 'Brand#45':Varchar) AND Not(Like(part.p_type, 'SMALL PLATED%':Varchar)) AND In(part.p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } - └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 13 } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 13 } ├── Upstream └── BatchPlanNode Fragment 6 StreamProject { exprs: [supplier.s_suppkey] } └── StreamFilter { predicate: Like(supplier.s_comment, '%Customer%Complaints%':Varchar) } - └── Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 14 } + └── StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 14 } ├── Upstream └── BatchPlanNode @@ -3574,7 +3563,7 @@ └── StreamExchange Hash([0]) from 4 Fragment 3 - Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── state table: 9 ├── Upstream └── BatchPlanNode @@ -3582,7 +3571,7 @@ Fragment 4 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 10 } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 10 } ├── Upstream └── BatchPlanNode @@ -3591,13 +3580,13 @@ └── StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } { intermediate state table: 16, state tables: [], distinct tables: [] } └── StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 17 } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 17 } ├── Upstream └── BatchPlanNode Fragment 6 StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } - └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 18 } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 18 } ├── Upstream └── BatchPlanNode @@ -3798,22 +3787,22 @@ └── StreamExchange Hash([1]) from 4 Fragment 3 - Chain { table: customer, columns: [customer.c_custkey, customer.c_name], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 15 } + StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 15 } ├── Upstream └── BatchPlanNode Fragment 4 - Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_totalprice, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 16 } + StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_totalprice, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 16 } ├── Upstream └── BatchPlanNode Fragment 5 - Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 17 } + StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 17 } ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 19 } + StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 19 } ├── Upstream └── BatchPlanNode @@ -3981,13 +3970,13 @@ Fragment 2 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: In(lineitem.l_shipmode, 'AIR':Varchar, 'AIR REG':Varchar) AND (lineitem.l_shipinstruct = 'DELIVER IN PERSON':Varchar) } - └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipinstruct, lineitem.l_shipmode], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 5 } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipinstruct, lineitem.l_shipmode], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 StreamFilter { predicate: (part.p_size >= 1:Int32) } - └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_size, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 6 } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_size, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -4128,7 +4117,7 @@ └── StreamExchange Hash([0]) from 4 Fragment 3 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── state table: 10 ├── Upstream └── BatchPlanNode @@ -4136,7 +4125,7 @@ Fragment 4 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'KENYA':Varchar) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 11 } + └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 11 } ├── Upstream └── BatchPlanNode @@ -4167,7 +4156,7 @@ Fragment 7 StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr1] } - └── Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── state table: 20 ├── Upstream └── BatchPlanNode @@ -4175,7 +4164,7 @@ Fragment 8 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } - └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── state table: 22 ├── Upstream └── BatchPlanNode @@ -4183,7 +4172,7 @@ Fragment 9 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: Like(part.p_name, 'forest%':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 23 } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 23 } ├── Upstream └── BatchPlanNode @@ -4425,12 +4414,12 @@ Fragment 5 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'GERMANY':Varchar) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 19 } + └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 19 } ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 20 } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 20 } ├── Upstream └── BatchPlanNode @@ -4442,27 +4431,27 @@ Fragment 8 StreamProject { exprs: [orders.o_orderkey] } └── StreamFilter { predicate: (orders.o_orderstatus = 'F':Varchar) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 25 } + └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 25 } ├── Upstream └── BatchPlanNode Fragment 9 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── state table: 26 ├── Upstream └── BatchPlanNode Fragment 10 - Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 27 } + StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 27 } ├── Upstream └── BatchPlanNode Fragment 11 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── state table: 28 ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/union.yaml b/src/frontend/planner_test/tests/testdata/output/union.yaml index 14e7b7e65cb70..725aab00d5a32 100644 --- a/src/frontend/planner_test/tests/testdata/output/union.yaml +++ b/src/frontend/planner_test/tests/testdata/output/union.yaml @@ -28,13 +28,13 @@ Fragment 1 StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, 0:Int32] } - └── Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 0 } + └── StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 2 StreamProject { exprs: [t2.a, t2.b, t2.c, t2._row_id, 1:Int32] } - └── Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 1 } + └── StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -109,14 +109,14 @@ Fragment 2 StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, 0:Int32] } - └── Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └── StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [t2.a, t2.b, t2.c, t2._row_id, 1:Int32] } - └── Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └── StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -199,14 +199,14 @@ Fragment 2 StreamProject { exprs: [t1.a, t1.b, t1.c, 0:Int32] } - └── Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } + └── StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } ├── state table: 1 ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [t2.a, t2.b, t2.c, 1:Int32] } - └── Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } + └── StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -360,31 +360,31 @@ Fragment 1 StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, 0:Int32] } - └── Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 0 } + └── StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 2 StreamProject { exprs: [t2.a, t2.b, t2.c, t2._row_id, 1:Int32] } - └── Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 1 } + └── StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 1 } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [t3.a, t3.b, t3.c, t3._row_id, 2:Int32] } - └── Chain { table: t3, columns: [t3.a, t3.b, t3.c, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } { state table: 2 } + └── StreamTableScan { table: t3, columns: [t3.a, t3.b, t3.c, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } { state table: 2 } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [t4.a, t4.b, t4.c, t4._row_id, 3:Int32] } - └── Chain { table: t4, columns: [t4.a, t4.b, t4.c, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } { state table: 3 } + └── StreamTableScan { table: t4, columns: [t4.a, t4.b, t4.c, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } { state table: 3 } ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [t5.a, t5.b, t5.c, t5._row_id, 4:Int32] } - └── Chain { table: t5, columns: [t5.a, t5.b, t5.c, t5._row_id], pk: [t5._row_id], dist: UpstreamHashShard(t5._row_id) } { state table: 4 } + └── StreamTableScan { table: t5, columns: [t5.a, t5.b, t5.c, t5._row_id], pk: [t5._row_id], dist: UpstreamHashShard(t5._row_id) } { state table: 4 } ├── Upstream └── BatchPlanNode @@ -456,31 +456,31 @@ Fragment 1 StreamProject { exprs: [t1.a, t1.b, t1.c, t1.a, null:Int64, null:Decimal, null:Serial, 0:Int32] } - └── Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 0 } + └── StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 2 StreamProject { exprs: [t2.a, t2.b, t2.c, null:Int32, null:Int64, t2.b, null:Serial, 1:Int32] } - └── Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.b], dist: UpstreamHashShard(t2.b) } { state table: 1 } + └── StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.b], dist: UpstreamHashShard(t2.b) } { state table: 1 } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [t3.a, t3.b, t3.c, null:Int32, t3.c, null:Decimal, null:Serial, 2:Int32] } - └── Chain { table: t3, columns: [t3.a, t3.b, t3.c], pk: [t3.c], dist: UpstreamHashShard(t3.c) } { state table: 2 } + └── StreamTableScan { table: t3, columns: [t3.a, t3.b, t3.c], pk: [t3.c], dist: UpstreamHashShard(t3.c) } { state table: 2 } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [t4.a, t4.b, t4.c, null:Int32, null:Int64, null:Decimal, t4._row_id, 3:Int32] } - └── Chain { table: t4, columns: [t4.a, t4.b, t4.c, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } { state table: 3 } + └── StreamTableScan { table: t4, columns: [t4.a, t4.b, t4.c, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } { state table: 3 } ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [t5.a, t5.b, t5.c, t5.a, null:Int64, t5.b, null:Serial, 4:Int32] } - └── Chain { table: t5, columns: [t5.a, t5.b, t5.c], pk: [t5.a, t5.b], dist: UpstreamHashShard(t5.a, t5.b) } { state table: 4 } + └── StreamTableScan { table: t5, columns: [t5.a, t5.b, t5.c], pk: [t5.a, t5.b], dist: UpstreamHashShard(t5.a, t5.b) } { state table: 4 } ├── Upstream └── BatchPlanNode @@ -517,31 +517,31 @@ Fragment 1 StreamProject { exprs: [t1.a, t1.b, t1.c, t1.a, null:Decimal, null:Serial, 0:Int32] } - └── Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 0 } + └── StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 2 StreamProject { exprs: [t2.a, t2.b, t2.c, null:Int32, t2.b, null:Serial, 1:Int32] } - └── Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.b], dist: UpstreamHashShard(t2.b) } { state table: 1 } + └── StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.b], dist: UpstreamHashShard(t2.b) } { state table: 1 } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [t3.a, t3.b, t3.c, null:Int32, null:Decimal, t3._row_id, 2:Int32] } - └── Chain { table: t3, columns: [t3.a, t3.b, t3.c, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } { state table: 2 } + └── StreamTableScan { table: t3, columns: [t3.a, t3.b, t3.c, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } { state table: 2 } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [t4.a, t4.b, t4.c, null:Int32, null:Decimal, t4._row_id, 3:Int32] } - └── Chain { table: t4, columns: [t4.a, t4.b, t4.c, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } { state table: 3 } + └── StreamTableScan { table: t4, columns: [t4.a, t4.b, t4.c, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } { state table: 3 } ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [t5.a, t5.b, t5.c, t5.a, t5.b, null:Serial, 4:Int32] } - └── Chain { table: t5, columns: [t5.a, t5.b, t5.c], pk: [t5.a, t5.b], dist: UpstreamHashShard(t5.a, t5.b) } { state table: 4 } + └── StreamTableScan { table: t5, columns: [t5.a, t5.b, t5.c], pk: [t5.a, t5.b], dist: UpstreamHashShard(t5.a, t5.b) } { state table: 4 } ├── Upstream └── BatchPlanNode @@ -578,31 +578,33 @@ Fragment 1 StreamProject { exprs: [t1.a, t1.b, t1.c, 0:Int32] } - └── Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 0 } + └── StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 2 StreamProject { exprs: [t2.a, t2.b, t2.c, 1:Int32] } - └── Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.b], dist: UpstreamHashShard(t2.b) } { state table: 1 } + └── StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.b], dist: UpstreamHashShard(t2.b) } { state table: 1 } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [t3.a, t3.b, t3.c, 2:Int32] } - └── Chain { table: t3, columns: [t3.a, t3.b, t3.c], pk: [t3.b], dist: UpstreamHashShard(t3.b) } { state table: 2 } + └── StreamTableScan { table: t3, columns: [t3.a, t3.b, t3.c], pk: [t3.b], dist: UpstreamHashShard(t3.b) } { state table: 2 } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [t4.a, t4.b, t4.c, 3:Int32] } - └── Chain { table: t4, columns: [t4.a, t4.b, t4.c], pk: [t4.b, t4.a], dist: UpstreamHashShard(t4.a, t4.b) } { state table: 3 } + └── StreamTableScan { table: t4, columns: [t4.a, t4.b, t4.c], pk: [t4.b, t4.a], dist: UpstreamHashShard(t4.a, t4.b) } + ├── state table: 3 ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [t5.a, t5.b, t5.c, 4:Int32] } - └── Chain { table: t5, columns: [t5.a, t5.b, t5.c], pk: [t5.a, t5.b], dist: UpstreamHashShard(t5.a, t5.b) } { state table: 4 } + └── StreamTableScan { table: t5, columns: [t5.a, t5.b, t5.c], pk: [t5.a, t5.b], dist: UpstreamHashShard(t5.a, t5.b) } + ├── state table: 4 ├── Upstream └── BatchPlanNode diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs index a4eda730d8941..6c20cc33ef2dd 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs @@ -109,10 +109,10 @@ mod tests { #[test] fn test_extract_mask() { - let mask = (FragmentTypeFlag::Source as u32) | (FragmentTypeFlag::ChainNode as u32); + let mask = (FragmentTypeFlag::Source as u32) | (FragmentTypeFlag::StreamScan as u32); let result = SysCatalogReaderImpl::extract_fragment_type_flag(mask); assert_eq!(result.len(), 2); assert!(result.contains(&FragmentTypeFlag::Source)); - assert!(result.contains(&FragmentTypeFlag::ChainNode)) + assert!(result.contains(&FragmentTypeFlag::StreamScan)) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 35a15a5d64792..a916700ab49a9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -19,7 +19,7 @@ use itertools::{EitherOrBoth, Itertools}; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::plan_common::JoinType; -use risingwave_pb::stream_plan::ChainType; +use risingwave_pb::stream_plan::StreamScanType; use super::generic::{ push_down_into_join, push_down_join_condition, GenericPlanNode, GenericPlanRef, @@ -1163,7 +1163,7 @@ impl LogicalJoin { .collect_vec(); // Use UpstreamOnly chain type let new_stream_table_scan = - StreamTableScan::new_with_chain_type(new_scan, ChainType::UpstreamOnly); + StreamTableScan::new_with_stream_scan_type(new_scan, StreamScanType::UpstreamOnly); let right = RequiredDist::no_shuffle(new_stream_table_scan.into()); // Construct a new logical join, because we have change its RHS. diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index bde0eeba44524..403f67e3c3869 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -21,7 +21,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, TableDesc}; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::util::sort_util::ColumnOrder; -use risingwave_pb::stream_plan::ChainType; +use risingwave_pb::stream_plan::StreamScanType; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::{childless_record, Distill}; @@ -566,10 +566,11 @@ impl ToStream for LogicalScan { } if self.predicate().always_true() { if self.is_cdc_table() { - Ok( - StreamTableScan::new_with_chain_type(self.core.clone(), ChainType::CdcBackfill) - .into(), + Ok(StreamTableScan::new_with_stream_scan_type( + self.core.clone(), + StreamScanType::CdcBackfill, ) + .into()) } else { Ok(StreamTableScan::new(self.core.clone()).into()) } diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 3979aceae757e..0fa89b87e1848 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -22,7 +22,7 @@ use risingwave_common::hash::VirtualNode; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use risingwave_pb::stream_plan::{ChainType, PbStreamNode}; +use risingwave_pb::stream_plan::{PbStreamNode, StreamScanType}; use super::stream::prelude::*; use super::utils::{childless_record, Distill}; @@ -36,22 +36,25 @@ use crate::stream_fragmenter::BuildFragmentGraphState; use crate::{Explain, TableCatalog}; /// `StreamTableScan` is a virtual plan node to represent a stream table scan. It will be converted -/// to chain + merge node (for upstream materialize) + batch table scan when converting to `MView` +/// to stream scan + merge node (for upstream materialize) + batch table scan when converting to `MView` /// creation request. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTableScan { pub base: PlanBase, core: generic::Scan, batch_plan_id: PlanNodeId, - chain_type: ChainType, + stream_scan_type: StreamScanType, } impl StreamTableScan { pub fn new(core: generic::Scan) -> Self { - Self::new_with_chain_type(core, ChainType::Backfill) + Self::new_with_stream_scan_type(core, StreamScanType::Backfill) } - pub fn new_with_chain_type(core: generic::Scan, chain_type: ChainType) -> Self { + pub fn new_with_stream_scan_type( + core: generic::Scan, + stream_scan_type: StreamScanType, + ) -> Self { let batch_plan_id = core.ctx.next_plan_node_id(); // TODO: correctly derive the distribution for cdc backfill @@ -80,7 +83,7 @@ impl StreamTableScan { base, core, batch_plan_id, - chain_type, + stream_scan_type, } } @@ -98,7 +101,7 @@ impl StreamTableScan { index_table_desc: Rc, primary_to_secondary_mapping: &BTreeMap, function_mapping: &HashMap, - chain_type: ChainType, + stream_scan_type: StreamScanType, ) -> StreamTableScan { let logical_index_scan = self.core.to_index_scan( index_name, @@ -109,11 +112,11 @@ impl StreamTableScan { logical_index_scan .distribution_key() .expect("distribution key of stream chain must exist in output columns"); - StreamTableScan::new_with_chain_type(logical_index_scan, chain_type) + StreamTableScan::new_with_stream_scan_type(logical_index_scan, stream_scan_type) } - pub fn chain_type(&self) -> ChainType { - self.chain_type + pub fn stream_scan_type(&self) -> StreamScanType { + self.stream_scan_type } /// Build catalog for backfill state @@ -240,16 +243,18 @@ impl StreamTableScan { .collect_vec(); // A flag to mark whether the upstream is a cdc source job - let cdc_upstream = matches!(self.chain_type, ChainType::CdcBackfill); + let cdc_upstream = matches!(self.stream_scan_type, StreamScanType::CdcBackfill); // The required columns from the table (both scan and upstream). - let upstream_column_ids = match self.chain_type { + let upstream_column_ids = match self.stream_scan_type { // For backfill, we additionally need the primary key columns. - ChainType::Backfill | ChainType::CdcBackfill => self.core.output_and_pk_column_ids(), - ChainType::Chain | ChainType::Rearrange | ChainType::UpstreamOnly => { + StreamScanType::Backfill | StreamScanType::CdcBackfill => { + self.core.output_and_pk_column_ids() + } + StreamScanType::Chain | StreamScanType::Rearrange | StreamScanType::UpstreamOnly => { self.core.output_column_ids() } - ChainType::ChainUnspecified => unreachable!(), + StreamScanType::ChainUnspecified => unreachable!(), } .iter() .map(ColumnId::get_id) @@ -309,9 +314,9 @@ impl StreamTableScan { let node_body = if cdc_upstream { // don't need batch plan for cdc source - PbNodeBody::Chain(ChainNode { + PbNodeBody::StreamScan(StreamScanNode { table_id: self.core.cdc_table_desc.table_id.table_id, - chain_type: self.chain_type as i32, + stream_scan_type: self.stream_scan_type as i32, // The column indices need to be forwarded to the downstream output_indices, upstream_column_ids, @@ -322,9 +327,9 @@ impl StreamTableScan { ..Default::default() }) } else { - PbNodeBody::Chain(ChainNode { + PbNodeBody::StreamScan(StreamScanNode { table_id: self.core.table_desc.table_id.table_id, - chain_type: self.chain_type as i32, + stream_scan_type: self.stream_scan_type as i32, // The column indices need to be forwarded to the downstream output_indices, upstream_column_ids, @@ -366,10 +371,7 @@ impl StreamTableScan { node_body: Some(node_body), stream_key, operator_id: self.base.id().0 as u64, - identity: { - let s = self.distill_to_string(); - s.replace("StreamTableScan", "Chain") - }, + identity: self.distill_to_string(), append_only: self.append_only(), } } @@ -383,6 +385,6 @@ impl ExprRewritable for StreamTableScan { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut core = self.core.clone(); core.rewrite_exprs(r); - Self::new_with_chain_type(core, self.chain_type).into() + Self::new_with_stream_scan_type(core, self.stream_scan_type).into() } } diff --git a/src/frontend/src/optimizer/rule/index_delta_join_rule.rs b/src/frontend/src/optimizer/rule/index_delta_join_rule.rs index 5ae42877f6454..8fb75b9c19d27 100644 --- a/src/frontend/src/optimizer/rule/index_delta_join_rule.rs +++ b/src/frontend/src/optimizer/rule/index_delta_join_rule.rs @@ -14,7 +14,7 @@ use itertools::Itertools; use risingwave_pb::plan_common::JoinType; -use risingwave_pb::stream_plan::ChainType; +use risingwave_pb::stream_plan::StreamScanType; use super::super::plan_node::*; use super::{BoxedRule, Rule}; @@ -52,7 +52,7 @@ impl Rule for IndexDeltaJoinRule { fn match_indexes( join_indices: &[usize], table_scan: &StreamTableScan, - chain_type: ChainType, + stream_scan_type: StreamScanType, ) -> Option { for index in &table_scan.core().indexes { // Only full covering index can be used in delta join @@ -96,7 +96,7 @@ impl Rule for IndexDeltaJoinRule { index.index_table.table_desc().into(), p2s_mapping, index.function_mapping(), - chain_type, + stream_scan_type, ) .into(), ); @@ -120,9 +120,9 @@ impl Rule for IndexDeltaJoinRule { return None; } - if chain_type != table_scan.chain_type() { + if stream_scan_type != table_scan.stream_scan_type() { Some( - StreamTableScan::new_with_chain_type(table_scan.core().clone(), chain_type) + StreamTableScan::new_with_stream_scan_type(table_scan.core().clone(), stream_scan_type) .into(), ) } else { @@ -136,8 +136,9 @@ impl Rule for IndexDeltaJoinRule { // Delta join only needs to backfill one stream flow and others should be upstream only // chain. Here we choose the left one to backfill and right one to upstream only // chain. - if let Some(left) = match_indexes(&left_indices, input_left, ChainType::Backfill) { - if let Some(right) = match_indexes(&right_indices, input_right, ChainType::UpstreamOnly) + if let Some(left) = match_indexes(&left_indices, input_left, StreamScanType::Backfill) { + if let Some(right) = + match_indexes(&right_indices, input_right, StreamScanType::UpstreamOnly) { // We already ensured that index and join use the same distribution, so we directly // replace the children with stream index scan without inserting any exchanges. diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index d762223e0b24d..887c55ecb84b3 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -135,7 +135,7 @@ fn is_stateful_executor(stream_node: &StreamNode) -> bool { NodeBody::HashAgg(_) | NodeBody::HashJoin(_) | NodeBody::DeltaIndexJoin(_) - | NodeBody::Chain(_) + | NodeBody::StreamScan(_) | NodeBody::DynamicFilter(_) ) } @@ -279,8 +279,8 @@ fn build_fragment( NodeBody::TopN(_) => current_fragment.requires_singleton = true, - NodeBody::Chain(node) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::ChainNode as u32; + NodeBody::StreamScan(node) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::StreamScan as u32; // memorize table id for later use // The table id could be a upstream CDC source state diff --git a/src/frontend/src/utils/stream_graph_formatter.rs b/src/frontend/src/utils/stream_graph_formatter.rs index f4a299f1aba4a..f55e9f6860894 100644 --- a/src/frontend/src/utils/stream_graph_formatter.rs +++ b/src/frontend/src/utils/stream_graph_formatter.rs @@ -263,7 +263,7 @@ impl StreamGraphFormatter { self.pretty_add_table(node.get_state_table().unwrap()), )); } - stream_node::NodeBody::Chain(node) => fields.push(( + stream_node::NodeBody::StreamScan(node) => fields.push(( "state table", self.pretty_add_table(node.get_state_table().unwrap()), )), diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index d428bbed31d8e..5ea20daccb85b 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -550,7 +550,7 @@ impl CommandContext { } } - /// For `CreateStreamingJob`, returns the actors of the `Chain` nodes. For other commands, + /// For `CreateStreamingJob`, returns the actors of the `StreamScan` nodes. For other commands, /// returns an empty set. pub fn actors_to_track(&self) -> HashSet { match &self.command { @@ -568,11 +568,11 @@ impl CommandContext { } } - /// For `CancelStreamingJob`, returns the actors of the `Chain` nodes. For other commands, + /// For `CancelStreamingJob`, returns the actors of the `StreamScan` nodes. For other commands, /// returns an empty set. pub fn actors_to_cancel(&self) -> HashSet { match &self.command { - Command::CancelStreamingJob(table_fragments) => table_fragments.chain_actor_ids(), + Command::CancelStreamingJob(table_fragments) => table_fragments.backfill_actor_ids(), _ => Default::default(), } } diff --git a/src/meta/src/barrier/notifier.rs b/src/meta/src/barrier/notifier.rs index b28c5b01d53d9..186fd501ac2a3 100644 --- a/src/meta/src/barrier/notifier.rs +++ b/src/meta/src/barrier/notifier.rs @@ -67,7 +67,7 @@ impl Notifier { /// /// Generally when a barrier is collected, it's also finished since it does not require further /// report of finishing from actors. - /// However for creating MV, this is only called when all `Chain` report it finished. + /// However for creating MV, this is only called when all `BackfillExecutor` report it finished. pub fn notify_finished(self) { if let Some(tx) = self.finished { tx.send(()).ok(); diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 2d5a745eaef5e..6b84ca238f1d7 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -36,16 +36,16 @@ use crate::MetaResult; type ConsumedRows = u64; #[derive(Clone, Copy, Debug)] -pub enum ChainState { +enum BackfillState { Init, ConsumingUpstream(Epoch, ConsumedRows), Done(ConsumedRows), } -/// Progress of all actors containing chain nodes while creating mview. +/// Progress of all actors containing backfill executors while creating mview. #[derive(Debug)] struct Progress { - states: HashMap, + states: HashMap, done_count: usize, @@ -54,7 +54,7 @@ struct Progress { /// appears in this stream job. upstream_mv_count: HashMap, - /// Upstream mvs total key count. + /// Total key count in the upstream materialized view upstream_total_key_count: u64, /// Consumed rows @@ -65,7 +65,7 @@ struct Progress { } impl Progress { - /// Create a [`Progress`] for some creating mview, with all `actors` containing the chain nodes. + /// Create a [`Progress`] for some creating mview, with all `actors` containing the backfill executors. fn new( actors: impl IntoIterator, upstream_mv_count: HashMap, @@ -74,7 +74,7 @@ impl Progress { ) -> Self { let states = actors .into_iter() - .map(|a| (a, ChainState::Init)) + .map(|a| (a, BackfillState::Init)) .collect::>(); assert!(!states.is_empty()); @@ -89,21 +89,21 @@ impl Progress { } /// Update the progress of `actor`. - fn update(&mut self, actor: ActorId, new_state: ChainState, upstream_total_key_count: u64) { + fn update(&mut self, actor: ActorId, new_state: BackfillState, upstream_total_key_count: u64) { self.upstream_total_key_count = upstream_total_key_count; match self.states.remove(&actor).unwrap() { - ChainState::Init => {} - ChainState::ConsumingUpstream(_, old_consumed_rows) => { + BackfillState::Init => {} + BackfillState::ConsumingUpstream(_, old_consumed_rows) => { self.consumed_rows -= old_consumed_rows; } - ChainState::Done(_) => panic!("should not report done multiple times"), + BackfillState::Done(_) => panic!("should not report done multiple times"), }; match &new_state { - ChainState::Init => {} - ChainState::ConsumingUpstream(_, new_consumed_rows) => { + BackfillState::Init => {} + BackfillState::ConsumingUpstream(_, new_consumed_rows) => { self.consumed_rows += new_consumed_rows; } - ChainState::Done(new_consumed_rows) => { + BackfillState::Done(new_consumed_rows) => { self.consumed_rows += new_consumed_rows; self.done_count += 1; } @@ -112,12 +112,12 @@ impl Progress { self.calculate_progress(); } - /// Returns whether all chains are done. + /// Returns whether all backfill executors are done. fn is_done(&self) -> bool { self.done_count == self.states.len() } - /// Returns the ids of all actors containing the chain nodes for the mview tracked by this + /// Returns the ids of all actors containing the backfill executors for the mview tracked by this /// [`Progress`]. fn actors(&self) -> impl Iterator + '_ { self.states.keys().cloned() @@ -240,7 +240,7 @@ pub(super) struct CreateMviewProgressTracker { /// Progress of the create-mview DDL indicated by the TableId. progress_map: HashMap, - /// Find the epoch of the create-mview DDL by the actor containing the chain node. + /// Find the epoch of the create-mview DDL by the actor containing the backfill executors. actor_map: HashMap, } @@ -265,11 +265,11 @@ impl CreateMviewProgressTracker { let mut progress_map = HashMap::new(); let table_map: HashMap<_, Vec> = table_map.into(); for (creating_table_id, actors) in table_map { - // 1. Recover `ChainState` in the tracker. + // 1. Recover `BackfillState` in the tracker. let mut states = HashMap::new(); for actor in actors { actor_map.insert(actor, creating_table_id); - states.insert(actor, ChainState::ConsumingUpstream(Epoch(0), 0)); + states.insert(actor, BackfillState::ConsumingUpstream(Epoch(0), 0)); } let upstream_mv_count = upstream_mv_counts.remove(&creating_table_id).unwrap(); let upstream_total_key_count = upstream_mv_count @@ -423,7 +423,7 @@ impl CreateMviewProgressTracker { progress: &CreateMviewProgress, version_stats: &HummockVersionStats, ) -> Option { - let actor = progress.chain_actor_id; + let actor = progress.backfill_actor_id; let Some(table_id) = self.actor_map.get(&actor).copied() else { // On restart, backfill will ALWAYS notify CreateMviewProgressTracker, // even if backfill is finished on recovery. @@ -439,9 +439,9 @@ impl CreateMviewProgressTracker { }; let new_state = if progress.done { - ChainState::Done(progress.consumed_rows) + BackfillState::Done(progress.consumed_rows) } else { - ChainState::ConsumingUpstream(progress.consumed_epoch.into(), progress.consumed_rows) + BackfillState::ConsumingUpstream(progress.consumed_epoch.into(), progress.consumed_rows) }; match self.progress_map.entry(table_id) { diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 051b3492cf68f..2df089e57667d 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -585,7 +585,7 @@ impl FragmentManager { let mut table_fragments = BTreeMapTransaction::new(map); for table_fragment in &to_delete_table_fragments { table_fragments.remove(table_fragment.table_id()); - let chain_actor_ids = table_fragment.chain_actor_ids(); + let backfill_actor_ids = table_fragment.backfill_actor_ids(); let dependent_table_ids = table_fragment.dependent_table_ids(); for (dependent_table_id, _) in dependent_table_ids { if table_ids.contains(&dependent_table_id) { @@ -608,7 +608,7 @@ impl FragmentManager { .for_each(|a| { a.dispatcher.retain_mut(|d| { d.downstream_actor_id - .retain(|x| !chain_actor_ids.contains(x)); + .retain(|x| !backfill_actor_ids.contains(x)); !d.downstream_actor_id.is_empty() }) }); @@ -1037,7 +1037,7 @@ impl FragmentManager { for (upstream_fragment_id, dispatcher_id) in upstream_fragment_dispatcher_ids { // here we assume the upstream fragment is in the same streaming job as this // fragment. Cross-table references only occur in the case - // of Chain fragment, and the scale of Chain fragment does not introduce updates + // of StreamScan fragment, and the scale of StreamScan fragment does not introduce updates // to the upstream Fragment (because of NoShuffle) let upstream_fragment = table_fragment .fragments @@ -1202,8 +1202,8 @@ impl FragmentManager { Ok(fragments) } - /// Get the downstream `Chain` fragments of the specified table. - pub async fn get_downstream_chain_fragments( + /// Get the downstream `StreamTableScan` fragments of the specified MV. + pub async fn get_downstream_fragments( &self, table_id: TableId, ) -> MetaResult> { @@ -1242,7 +1242,7 @@ impl FragmentManager { .map(|d| (d.clone(), fragment.clone())) }) .inspect(|(_, f)| { - assert!((f.fragment_type_mask & FragmentTypeFlag::ChainNode as u32) != 0) + assert!((f.fragment_type_mask & FragmentTypeFlag::StreamScan as u32) != 0) }) }) .collect_vec(); diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 3026880b1da09..aba928fecde41 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -313,10 +313,10 @@ impl TableFragments { .cloned() } - /// Returns actors that contains Chain node. - pub fn chain_actor_ids(&self) -> HashSet { + /// Returns actors that contains backfill executors. + pub fn backfill_actor_ids(&self) -> HashSet { Self::filter_actor_ids(self, |fragment_type_mask| { - (fragment_type_mask & FragmentTypeFlag::ChainNode as u32) != 0 + (fragment_type_mask & FragmentTypeFlag::StreamScan as u32) != 0 }) .into_iter() .collect() @@ -364,9 +364,9 @@ impl TableFragments { /// Resolve dependent table fn resolve_dependent_table(stream_node: &StreamNode, table_ids: &mut HashMap) { - if let Some(NodeBody::Chain(chain)) = stream_node.node_body.as_ref() { + if let Some(NodeBody::StreamScan(stream_scan)) = stream_node.node_body.as_ref() { table_ids - .entry(TableId::new(chain.table_id)) + .entry(TableId::new(stream_scan.table_id)) .or_default() .add_assign(1); } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 38ed3f16ae37f..0cd3af7385184 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -1009,7 +1009,7 @@ impl DdlController { // Map the column indices in the dispatchers with the given mapping. let downstream_fragments = self .fragment_manager - .get_downstream_chain_fragments(id.into()) + .get_downstream_fragments(id.into()) .await? .into_iter() .map(|(d, f)| Some((table_col_index_mapping.rewrite_dispatch_strategy(&d)?, f))) diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index 7cd208496954d..22f0740fc5c0f 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -26,7 +26,8 @@ use risingwave_pb::meta::table_fragments::Fragment; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::update_mutation::MergeUpdate; use risingwave_pb::stream_plan::{ - ChainType, DispatchStrategy, Dispatcher, DispatcherType, MergeNode, StreamActor, StreamNode, + DispatchStrategy, Dispatcher, DispatcherType, MergeNode, StreamActor, StreamNode, + StreamScanType, }; use super::id::GlobalFragmentIdsExt; @@ -120,7 +121,7 @@ impl ActorBuilder { /// During this process, the following things will be done: /// 1. Replace the logical `Exchange` in node's input with `Merge`, which can be executed on the /// compute nodes. - /// 2. Fill the upstream mview info of the `Merge` node under the `Chain` node. + /// 2. Fill the upstream mview info of the `Merge` node under the `StreamScan` node. fn rewrite(&self) -> MetaResult { self.rewrite_inner(&self.nodes, 0) } @@ -157,9 +158,10 @@ impl ActorBuilder { }) } - // "Leaf" node `Chain`. - NodeBody::Chain(chain_node) => { - let cdc_backfill = chain_node.chain_type == ChainType::CdcBackfill as i32; + // "Leaf" node `StreamScan`. + NodeBody::StreamScan(stream_scan) => { + let cdc_backfill = + stream_scan.stream_scan_type == StreamScanType::CdcBackfill as i32; let input = stream_node.get_input(); assert_eq!(input.len(), 2); @@ -171,7 +173,7 @@ impl ActorBuilder { // Index the upstreams by the an external edge ID. let upstreams = &self.upstreams[&EdgeId::UpstreamExternal { - upstream_table_id: chain_node.table_id.into(), + upstream_table_id: stream_scan.table_id.into(), downstream_fragment_id: self.fragment_id, }]; @@ -180,7 +182,7 @@ impl ActorBuilder { let upstream_actor_id = upstreams.actors.as_global_ids(); assert_eq!(upstream_actor_id.len(), 1); - let chain_input = vec![ + let input = vec![ // Fill the merge node body with correct upstream info. StreamNode { node_body: Some(NodeBody::Merge(MergeNode { @@ -199,7 +201,7 @@ impl ActorBuilder { ]; Ok(StreamNode { - input: chain_input, + input, ..stream_node.clone() }) } diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 19641a06e5d85..29b61e1056f1e 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -170,9 +170,9 @@ impl BuildingFragment { let mut table_columns = HashMap::new(); stream_graph_visitor::visit_fragment(fragment, |node_body| { - if let NodeBody::Chain(chain_node) = node_body { - let table_id = chain_node.table_id.into(); - let column_ids = chain_node.upstream_column_ids.clone(); + if let NodeBody::StreamScan(stream_scan) = node_body { + let table_id = stream_scan.table_id.into(); + let column_ids = stream_scan.upstream_column_ids.clone(); table_columns .try_insert(table_id, column_ids) .expect("currently there should be no two same upstream tables in a fragment"); @@ -466,7 +466,7 @@ pub(super) enum EitherFragment { /// - if we're going to build a mview on an existing mview, the upstream fragment containing the /// `Materialize` node will be included in this structure. /// - if we're going to replace the plan of a table with downstream mviews, the downstream fragments -/// containing the `Chain` nodes will be included in this structure. +/// containing the `StreamScan` nodes will be included in this structure. pub struct CompleteStreamFragmentGraph { /// The fragment graph of the streaming job being built. building_graph: StreamFragmentGraph, @@ -523,7 +523,7 @@ impl CompleteStreamFragmentGraph { } /// Create a new [`CompleteStreamFragmentGraph`] for replacing an existing table, with the - /// downstream existing `Chain` fragments. + /// downstream existing `StreamScan` fragments. pub fn with_downstreams( graph: StreamFragmentGraph, original_table_fragment_id: FragmentId, @@ -554,7 +554,7 @@ impl CompleteStreamFragmentGraph { upstream_root_fragments, }) = upstream_ctx { - // Build the extra edges between the upstream `Materialize` and the downstream `Chain` + // Build the extra edges between the upstream `Materialize` and the downstream `StreamScan` // of the new materialized view. for (&id, fragment) in &mut graph.fragments { for (&upstream_table_id, output_columns) in &fragment.upstream_table_columns { @@ -563,8 +563,8 @@ impl CompleteStreamFragmentGraph { // extract the upstream full_table_name from the source fragment let mut full_table_name = None; visit_fragment(&mut fragment.inner, |node_body| { - if let NodeBody::Chain(chain_node) = node_body { - full_table_name = chain_node + if let NodeBody::StreamScan(stream_scan) = node_body { + full_table_name = stream_scan .cdc_table_desc .as_ref() .map(|desc| desc.table_name.clone()); @@ -593,7 +593,7 @@ impl CompleteStreamFragmentGraph { ?source_job_id, ?rw_table_name_index, ?output_columns, - "chain with upstream source fragment" + "StreamScan with upstream source fragment" ); let edge = StreamFragmentEdge { id: EdgeId::UpstreamExternal { @@ -647,7 +647,7 @@ impl CompleteStreamFragmentGraph { downstream_fragment_id: id, }, // We always use `NoShuffle` for the exchange between the upstream - // `Materialize` and the downstream `Chain` of the + // `Materialize` and the downstream `StreamScan` of the // new materialized view. dispatch_strategy: DispatchStrategy { r#type: DispatcherType::NoShuffle as _, @@ -690,7 +690,7 @@ impl CompleteStreamFragmentGraph { let original_table_fragment_id = GlobalFragmentId::new(original_table_fragment_id); let table_fragment_id = GlobalFragmentId::new(graph.table_fragment_id()); - // Build the extra edges between the `Materialize` and the downstream `Chain` of the + // Build the extra edges between the `Materialize` and the downstream `StreamScan` of the // existing materialized views. for (dispatch_strategy, fragment) in &downstream_fragments { let id = GlobalFragmentId::new(fragment.fragment_id); diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs index 11f428a09b71d..de69e72fbeb1d 100644 --- a/src/stream/src/from_proto/mod.rs +++ b/src/stream/src/from_proto/mod.rs @@ -18,7 +18,6 @@ mod agg_common; mod append_only_dedup; mod barrier_recv; mod batch_query; -mod chain; mod dml; mod dynamic_filter; mod eowc_over_window; @@ -43,6 +42,7 @@ mod sink; mod sort; mod source; mod stateless_simple_agg; +mod stream_scan; mod temporal_join; mod top_n; mod union; @@ -58,7 +58,6 @@ use risingwave_storage::StateStore; use self::append_only_dedup::*; use self::barrier_recv::*; use self::batch_query::*; -use self::chain::*; use self::dml::*; use self::dynamic_filter::*; use self::eowc_over_window::*; @@ -83,6 +82,7 @@ use self::sink::*; use self::sort::*; use self::source::*; use self::stateless_simple_agg::*; +use self::stream_scan::*; use self::temporal_join::*; use self::top_n::*; use self::union::*; @@ -139,7 +139,7 @@ pub async fn create_executor( NodeBody::HashAgg => HashAggExecutorBuilder, NodeBody::HashJoin => HashJoinExecutorBuilder, NodeBody::HopWindow => HopWindowExecutorBuilder, - NodeBody::Chain => ChainExecutorBuilder, + NodeBody::StreamScan => ChainExecutorBuilder, NodeBody::BatchPlan => BatchQueryExecutorBuilder, NodeBody::Merge => MergeExecutorBuilder, NodeBody::Materialize => MaterializeExecutorBuilder, diff --git a/src/stream/src/from_proto/chain.rs b/src/stream/src/from_proto/stream_scan.rs similarity index 92% rename from src/stream/src/from_proto/chain.rs rename to src/stream/src/from_proto/stream_scan.rs index 5e41328f5ff81..6b6247e9509cb 100644 --- a/src/stream/src/from_proto/chain.rs +++ b/src/stream/src/from_proto/stream_scan.rs @@ -20,7 +20,7 @@ use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema, TableId, TableOpt use risingwave_common::util::sort_util::OrderType; use risingwave_connector::source::external::{CdcTableType, SchemaTableName}; use risingwave_pb::plan_common::{ExternalTableDesc, StorageTableDesc}; -use risingwave_pb::stream_plan::{ChainNode, ChainType}; +use risingwave_pb::stream_plan::{StreamScanNode, StreamScanType}; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::Distribution; @@ -35,7 +35,7 @@ use crate::executor::{ pub struct ChainExecutorBuilder; impl ExecutorBuilder for ChainExecutorBuilder { - type Node = ChainNode; + type Node = StreamScanNode; async fn new_boxed_executor( params: ExecutorParams, @@ -55,14 +55,14 @@ impl ExecutorBuilder for ChainExecutorBuilder { .map(|&i| i as usize) .collect_vec(); - let schema = if matches!(node.chain_type(), ChainType::Backfill) { + let schema = if matches!(node.stream_scan_type(), StreamScanType::Backfill) { Schema::new( output_indices .iter() .map(|i| snapshot.schema().fields()[*i].clone()) .collect_vec(), ) - } else if matches!(node.chain_type(), ChainType::CdcBackfill) { + } else if matches!(node.stream_scan_type(), StreamScanType::CdcBackfill) { let table_desc: &ExternalTableDesc = node.get_cdc_table_desc()?; let schema = Schema::new(table_desc.columns.iter().map(Into::into).collect()); assert_eq!(output_indices, (0..schema.len()).collect_vec()); @@ -75,9 +75,9 @@ impl ExecutorBuilder for ChainExecutorBuilder { snapshot.schema().clone() }; - let executor = match node.chain_type() { - ChainType::Chain | ChainType::UpstreamOnly => { - let upstream_only = matches!(node.chain_type(), ChainType::UpstreamOnly); + let executor = match node.stream_scan_type() { + StreamScanType::Chain | StreamScanType::UpstreamOnly => { + let upstream_only = matches!(node.stream_scan_type(), StreamScanType::UpstreamOnly); ChainExecutor::new( snapshot, upstream, @@ -88,7 +88,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { ) .boxed() } - ChainType::Rearrange => RearrangedChainExecutor::new( + StreamScanType::Rearrange => RearrangedChainExecutor::new( snapshot, upstream, progress, @@ -96,7 +96,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { params.pk_indices, ) .boxed(), - ChainType::CdcBackfill => { + StreamScanType::CdcBackfill => { let table_desc: &ExternalTableDesc = node.get_cdc_table_desc()?; let properties: HashMap = table_desc .connect_properties @@ -149,7 +149,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { params.env.config().developer.chunk_size, ).boxed() } - ChainType::Backfill => { + StreamScanType::Backfill => { let table_desc: &StorageTableDesc = node .get_table_desc() .map_err(|err| anyhow!("chain: table_desc not found! {:?}", err))?; @@ -244,7 +244,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { ) .boxed() } - ChainType::ChainUnspecified => unreachable!(), + StreamScanType::ChainUnspecified => unreachable!(), }; let rate_limit = node.get_rate_limit().cloned().ok(); Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 43aeb4afba46b..9c1a02d43d0ca 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -22,7 +22,7 @@ use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgres use risingwave_storage::{dispatch_state_store, StateStore, StateStoreImpl}; use tokio::sync::oneshot; -use super::progress::ChainState; +use super::progress::BackfillState; use super::CollectResult; use crate::error::{StreamError, StreamResult}; use crate::executor::monitor::GLOBAL_STREAMING_METRICS; @@ -64,7 +64,7 @@ pub(super) struct ManagedBarrierState { epoch_barrier_state_map: BTreeMap, /// Record the progress updates of creating mviews for each epoch of concurrent checkpoints. - pub(super) create_mview_progress: HashMap>, + pub(super) create_mview_progress: HashMap>, /// Record all unexpected exited actors. failure_actors: HashMap, @@ -111,15 +111,15 @@ impl ManagedBarrierState { .unwrap_or_default() .into_iter() .map(|(actor, state)| CreateMviewProgress { - chain_actor_id: actor, - done: matches!(state, ChainState::Done(_)), + backfill_actor_id: actor, + done: matches!(state, BackfillState::Done(_)), consumed_epoch: match state { - ChainState::ConsumingUpstream(consumed_epoch, _) => consumed_epoch, - ChainState::Done(_) => epoch, + BackfillState::ConsumingUpstream(consumed_epoch, _) => consumed_epoch, + BackfillState::Done(_) => epoch, }, consumed_rows: match state { - ChainState::ConsumingUpstream(_, consumed_rows) => consumed_rows, - ChainState::Done(consumed_rows) => consumed_rows, + BackfillState::ConsumingUpstream(_, consumed_rows) => consumed_rows, + BackfillState::Done(consumed_rows) => consumed_rows, }, }) .collect(); diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index 5abeab216cd00..a1b74773232d0 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -21,7 +21,7 @@ type ConsumedEpoch = u64; type ConsumedRows = u64; #[derive(Debug, Clone, Copy)] -pub(super) enum ChainState { +pub(super) enum BackfillState { ConsumingUpstream(ConsumedEpoch, ConsumedRows), Done(ConsumedRows), } @@ -31,7 +31,7 @@ impl LocalBarrierManager { &mut self, current_epoch: u64, actor: ActorId, - state: ChainState, + state: BackfillState, ) { match &mut self.state { #[cfg(test)] @@ -48,7 +48,7 @@ impl LocalBarrierManager { } } -/// The progress held by the chain executors to report to the local barrier manager. +/// The progress held by the backfill executors to report to the local barrier manager. /// /// Progress can be computed by /// `total_rows_consumed` / `total_rows_upstream`. @@ -81,20 +81,20 @@ impl LocalBarrierManager { pub struct CreateMviewProgress { barrier_manager: Arc>, - /// The id of the actor containing the chain node. - chain_actor_id: ActorId, + /// The id of the actor containing the backfill executors. + backfill_actor_id: ActorId, - state: Option, + state: Option, } impl CreateMviewProgress { pub fn new( barrier_manager: Arc>, - chain_actor_id: ActorId, + backfill_actor_id: ActorId, ) -> Self { Self { barrier_manager, - chain_actor_id, + backfill_actor_id, state: None, } } @@ -105,14 +105,14 @@ impl CreateMviewProgress { } pub fn actor_id(&self) -> u32 { - self.chain_actor_id + self.backfill_actor_id } - fn update_inner(&mut self, current_epoch: u64, state: ChainState) { + fn update_inner(&mut self, current_epoch: u64, state: BackfillState) { self.state = Some(state); self.barrier_manager.lock().update_create_mview_progress( current_epoch, - self.chain_actor_id, + self.backfill_actor_id, state, ); } @@ -128,7 +128,7 @@ impl CreateMviewProgress { current_consumed_rows: ConsumedRows, ) { match self.state { - Some(ChainState::ConsumingUpstream(last, last_consumed_rows)) => { + Some(BackfillState::ConsumingUpstream(last, last_consumed_rows)) => { assert!( last < consumed_epoch, "last_epoch: {:#?} must be greater than consumed epoch: {:#?}", @@ -137,35 +137,38 @@ impl CreateMviewProgress { ); assert!(last_consumed_rows <= current_consumed_rows); } - Some(ChainState::Done(_)) => unreachable!(), + Some(BackfillState::Done(_)) => unreachable!(), None => {} }; self.update_inner( current_epoch, - ChainState::ConsumingUpstream(consumed_epoch, current_consumed_rows), + BackfillState::ConsumingUpstream(consumed_epoch, current_consumed_rows), ); } /// Finish the progress. If the progress is already finished, then perform no-op. /// `current_epoch` should be provided to locate the barrier under concurrent checkpoint. pub fn finish(&mut self, current_epoch: u64, current_consumed_rows: ConsumedRows) { - if let Some(ChainState::Done(_)) = self.state { + if let Some(BackfillState::Done(_)) = self.state { return; } - self.update_inner(current_epoch, ChainState::Done(current_consumed_rows)); + self.update_inner(current_epoch, BackfillState::Done(current_consumed_rows)); } } impl SharedContext { - /// Create a struct for reporting the progress of creating mview. The chain executors should + /// Create a struct for reporting the progress of creating mview. The backfill executors should /// report the progress of barrier rearranging continuously using this. The updated progress /// will be collected by the local barrier manager and reported to the meta service in this /// epoch. /// - /// When all chain executors of the creating mview finish, the creation progress will be done at + /// When all backfill executors of the creating mview finish, the creation progress will be done at /// frontend and the mview will be exposed to the user. - pub fn register_create_mview_progress(&self, chain_actor_id: ActorId) -> CreateMviewProgress { - trace!("register create mview progress: {}", chain_actor_id); - CreateMviewProgress::new(self.barrier_manager.clone(), chain_actor_id) + pub fn register_create_mview_progress( + &self, + backfill_actor_id: ActorId, + ) -> CreateMviewProgress { + trace!("register create mview progress: {}", backfill_actor_id); + CreateMviewProgress::new(self.barrier_manager.clone(), backfill_actor_id) } } diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 4386413029ae1..7ee45a4d24f5e 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -520,7 +520,7 @@ impl LocalStreamManagerCore { | NodeBody::HashJoin(_) | NodeBody::DeltaIndexJoin(_) | NodeBody::Lookup(_) - | NodeBody::Chain(_) + | NodeBody::StreamScan(_) | NodeBody::DynamicFilter(_) | NodeBody::GroupTopN(_) | NodeBody::Now(_) diff --git a/src/tests/simulation/tests/integration_tests/recovery/backfill.rs b/src/tests/simulation/tests/integration_tests/recovery/backfill.rs index 896f65506554e..a2c02e25de237 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/backfill.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/backfill.rs @@ -77,7 +77,7 @@ async fn test_snapshot_mv() -> Result<()> { let fragment = cluster .locate_one_fragment([ identity_contains("materialize"), - no_identity_contains("chain"), + no_identity_contains("StreamTableScan"), ]) .await?; @@ -124,7 +124,7 @@ async fn test_backfill_mv() -> Result<()> { let fragment = cluster .locate_one_fragment([ identity_contains("materialize"), - no_identity_contains("chain"), + no_identity_contains("StreamTableScan"), ]) .await?; @@ -175,7 +175,10 @@ async fn test_index_backfill() -> Result<()> { assert_eq!(results.lines().collect_vec().len(), 256); let fragment = cluster - .locate_one_fragment([identity_contains("index"), no_identity_contains("chain")]) + .locate_one_fragment([ + identity_contains("index"), + no_identity_contains("StreamTableScan"), + ]) .await?; let id = fragment.id(); diff --git a/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs b/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs index 776692b2fab90..f4c2f5010dd38 100644 --- a/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs +++ b/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs @@ -39,7 +39,7 @@ async fn test_simple_cascade_materialized_view() -> Result<()> { let fragment = cluster .locate_one_fragment([ identity_contains("materialize"), - no_identity_contains("chain"), + no_identity_contains("StreamTableScan"), no_identity_contains("topn"), no_identity_contains("hashjoin"), ]) @@ -54,7 +54,7 @@ async fn test_simple_cascade_materialized_view() -> Result<()> { assert_eq!(fragment.inner.actors.len(), 1); let chain_fragment = cluster - .locate_one_fragment([identity_contains("chain")]) + .locate_one_fragment([identity_contains("StreamTableScan")]) .await?; assert_eq!( @@ -84,7 +84,7 @@ async fn test_simple_cascade_materialized_view() -> Result<()> { assert_eq!(fragment.inner.actors.len(), 6); let chain_fragment = cluster - .locate_one_fragment([identity_contains("chain")]) + .locate_one_fragment([identity_contains("StreamTableScan")]) .await?; assert_eq!( @@ -129,7 +129,7 @@ async fn test_diamond_cascade_materialized_view() -> Result<()> { let fragment = cluster .locate_one_fragment([ identity_contains("materialize"), - no_identity_contains("chain"), + no_identity_contains("StreamTableScan"), no_identity_contains("topn"), no_identity_contains("hashjoin"), ]) diff --git a/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs b/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs index 0fde294d08e7f..6b04503ab2bc5 100644 --- a/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs +++ b/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs @@ -151,7 +151,7 @@ async fn test_resolve_no_shuffle_upstream() -> Result<()> { .await?; let fragment = cluster - .locate_one_fragment([identity_contains("chain")]) + .locate_one_fragment([identity_contains("StreamTableScan")]) .await?; let result = cluster.reschedule(fragment.reschedule([0], [])).await; diff --git a/src/tests/simulation/tests/integration_tests/scale/plan.rs b/src/tests/simulation/tests/integration_tests/scale/plan.rs index 8b62a58998a3f..22e61cdb316de 100644 --- a/src/tests/simulation/tests/integration_tests/scale/plan.rs +++ b/src/tests/simulation/tests/integration_tests/scale/plan.rs @@ -197,7 +197,10 @@ async fn test_resize_single_failed() -> Result<()> { let upstream_fragment_id = upstream_fragment.inner.fragment_id; let downstream_fragment = cluster - .locate_one_fragment([identity_contains("chain"), identity_contains("materialize")]) + .locate_one_fragment([ + identity_contains("StreamTableScan"), + identity_contains("materialize"), + ]) .await?; let downstream_fragment_id = downstream_fragment.inner.fragment_id; @@ -272,7 +275,7 @@ join mv5 on mv1.v = mv5.v limit 1;", .await?; let chain_fragments: [_; 8] = cluster - .locate_fragments([identity_contains("chain")]) + .locate_fragments([identity_contains("StreamTableScan")]) .await? .try_into() .unwrap(); @@ -314,7 +317,7 @@ join mv5 on mv1.v = mv5.v limit 1;", .locate_one_fragment([ identity_contains("materialize"), no_identity_contains("topn"), - no_identity_contains("chain"), + no_identity_contains("StreamTableScan"), no_identity_contains("hashJoin"), ]) .await?; From 901d9b0f54a10f9a5e26b895a6b765b98d27d007 Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 8 Nov 2023 16:22:20 +0800 Subject: [PATCH 11/77] refactor(optimizer): add lookup table to batch lookup join explain (#13311) --- .../testdata/output/batch_index_join.yaml | 14 ++-- .../tests/testdata/output/ch_benchmark.yaml | 76 +++++++++--------- .../output/distributed_lookup_join.yaml | 8 +- .../testdata/output/distribution_derive.yaml | 10 +-- .../tests/testdata/output/except.yaml | 2 +- .../testdata/output/index_selection.yaml | 78 +++++++++---------- .../tests/testdata/output/intersect.yaml | 2 +- .../tests/testdata/output/join.yaml | 12 +-- .../tests/testdata/output/nexmark.yaml | 4 +- .../tests/testdata/output/subquery.yaml | 2 +- .../tests/testdata/output/time_window.yaml | 4 +- .../tests/testdata/output/tpch.yaml | 72 ++++++++--------- .../optimizer/plan_node/batch_lookup_join.rs | 7 +- 13 files changed, 148 insertions(+), 143 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml b/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml index 2d1b0951089e8..72abd65027d04 100644 --- a/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml @@ -6,7 +6,7 @@ select * from t join t2 on t.b = t2.d; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: t.b = idx.d, output: all } + └─BatchLookupJoin { type: Inner, predicate: t.b = idx.d, output: all, lookup table: idx } └─BatchExchange { order: [], dist: UpstreamHashShard(t.b) } └─BatchScan { table: t, columns: [t.a, t.b], distribution: SomeShard } - sql: | @@ -18,7 +18,7 @@ select * from t join t2 on t.a = t2.c and t.b = t2.d; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: t.a = idx2.c AND t.b = idx2.d, output: all } + └─BatchLookupJoin { type: Inner, predicate: t.a = idx2.c AND t.b = idx2.d, output: all, lookup table: idx2 } └─BatchExchange { order: [], dist: UpstreamHashShard(t.a) } └─BatchScan { table: t, columns: [t.a, t.b], distribution: SomeShard } - sql: | @@ -28,7 +28,7 @@ select * from t join t2 on t.b = t2.d; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: t.b = idx.d, output: all } + └─BatchLookupJoin { type: Inner, predicate: t.b = idx.d, output: all, lookup table: idx } └─BatchExchange { order: [], dist: UpstreamHashShard(t.b) } └─BatchScan { table: t, columns: [t.a, t.b], distribution: SomeShard } - name: test index join predicate reorder @@ -39,7 +39,7 @@ select * from t join t2 on t.b = t2.d and t.a = t2.c; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: t.a = idx.c AND t.b = idx.d, output: all } + └─BatchLookupJoin { type: Inner, predicate: t.a = idx.c AND t.b = idx.d, output: all, lookup table: idx } └─BatchExchange { order: [], dist: UpstreamHashShard(t.a) } └─BatchScan { table: t, columns: [t.a, t.b], distribution: SomeShard } - name: test index join prefix lookup @@ -50,7 +50,7 @@ select * from t join t2 on t.a = t2.c; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: t.a = idx.c, output: all } + └─BatchLookupJoin { type: Inner, predicate: t.a = idx.c, output: all, lookup table: idx } └─BatchExchange { order: [], dist: UpstreamHashShard(t.a) } └─BatchScan { table: t, columns: [t.a, t.b], distribution: SomeShard } - name: test index join distribution derive @@ -63,7 +63,7 @@ └─BatchHashAgg { group_key: [internal_last_seen_value(t2.c)], aggs: [internal_last_seen_value(internal_last_seen_value(t2.d)), count(t.a)] } └─BatchExchange { order: [], dist: HashShard(internal_last_seen_value(t2.c)) } └─BatchHashAgg { group_key: [t.a], aggs: [internal_last_seen_value(t2.c), internal_last_seen_value(t2.d)] } - └─BatchLookupJoin { type: Inner, predicate: t.a = t2.c, output: [t2.c, t2.d, t.a] } + └─BatchLookupJoin { type: Inner, predicate: t.a = t2.c, output: [t2.c, t2.d, t.a], lookup table: t2 } └─BatchExchange { order: [], dist: UpstreamHashShard(t.a) } └─BatchScan { table: t, columns: [t.a], distribution: SomeShard } - sql: | @@ -73,7 +73,7 @@ select * from t1 join idx on t1.a = idx.c and t1.b = idx.d; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: t1.a = idx.c AND t1.b = idx.d, output: all } + └─BatchLookupJoin { type: Inner, predicate: t1.a = idx.c AND t1.b = idx.d, output: all, lookup table: idx } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } - name: shouldn't be a lookup join diff --git a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml index 7eed762be53bc..97e64b0867b8c 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -105,11 +105,11 @@ batch_plan: |- BatchExchange { order: [nation.n_name ASC, supplier.s_name ASC, item.i_id ASC], dist: Single } └─BatchSort { order: [nation.n_name ASC, supplier.s_name ASC, item.i_id ASC] } - └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name >= 'EUROP':Varchar) AND (region.r_name < 'EUROQ':Varchar), output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment] } + └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name >= 'EUROP':Varchar) AND (region.r_name < 'EUROQ':Varchar), output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment], lookup table: region } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [item.i_id, item.i_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [item.i_id, item.i_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [item.i_id, item.i_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment] } + └─BatchLookupJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [item.i_id, item.i_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard($expr2) } └─BatchProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2] } └─BatchHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id AND stock.s_quantity = min(stock.s_quantity), output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id] } @@ -124,9 +124,9 @@ └─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } └─BatchHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity] } ├─BatchExchange { order: [], dist: HashShard(supplier.s_suppkey) } - │ └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name >= 'EUROP':Varchar) AND (region.r_name < 'EUROQ':Varchar), output: [supplier.s_suppkey] } + │ └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name >= 'EUROP':Varchar) AND (region.r_name < 'EUROQ':Varchar), output: [supplier.s_suppkey], lookup table: region } │ └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } - │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey] } + │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey], lookup table: nation } │ └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } └─BatchExchange { order: [], dist: HashShard($expr1) } @@ -399,7 +399,7 @@ └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, orders.o_entry_d) } └─BatchHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, orders.o_entry_d, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = new_order.no_w_id AND orders.o_d_id = new_order.no_d_id AND orders.o_id = new_order.no_o_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } + │ └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = new_order.no_w_id AND orders.o_d_id = new_order.no_d_id AND orders.o_id = new_order.no_o_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], lookup table: new_order } │ └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─BatchHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } │ ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } @@ -654,14 +654,14 @@ BatchExchange { order: [], dist: Single } └─BatchHashAgg { group_key: [nation.n_name], aggs: [sum(order_line.ol_amount)] } └─BatchExchange { order: [], dist: HashShard(nation.n_name) } - └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'EUROPE':Varchar), output: [nation.n_name, order_line.ol_amount] } + └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'EUROPE':Varchar), output: [nation.n_name, order_line.ol_amount], lookup table: region } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [order_line.ol_amount, nation.n_name, nation.n_regionkey] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [order_line.ol_amount, nation.n_name, nation.n_regionkey], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey AND $expr2 = supplier.s_nationkey, output: [order_line.ol_amount, supplier.s_nationkey] } + └─BatchLookupJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey AND $expr2 = supplier.s_nationkey, output: [order_line.ol_amount, supplier.s_nationkey], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2] } - └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [customer.c_state, order_line.ol_amount, stock.s_i_id, stock.s_w_id] } + └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [customer.c_state, order_line.ol_amount, stock.s_i_id, stock.s_w_id], lookup table: stock } └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id, order_line.ol_w_id) } └─BatchHashJoin { type: Inner, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id, output: [customer.c_state, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } @@ -971,17 +971,17 @@ └─BatchHashAgg { group_key: [supplier.s_nationkey, $expr3, $expr4], aggs: [sum(order_line.ol_amount)] } └─BatchExchange { order: [], dist: HashShard(supplier.s_nationkey, $expr3, $expr4) } └─BatchProject { exprs: [supplier.s_nationkey, Substr(customer.c_state, 1:Int32, 1:Int32) as $expr3, Extract('YEAR':Varchar, orders.o_entry_d) as $expr4, order_line.ol_amount] } - └─BatchLookupJoin { type: Inner, predicate: $expr2 = nation.n_nationkey AND (((nation.n_name = 'JAPAN':Varchar) AND (nation.n_name = 'CHINA':Varchar)) OR ((nation.n_name = 'CHINA':Varchar) AND (nation.n_name = 'JAPAN':Varchar))), output: [supplier.s_nationkey, order_line.ol_amount, orders.o_entry_d, customer.c_state] } + └─BatchLookupJoin { type: Inner, predicate: $expr2 = nation.n_nationkey AND (((nation.n_name = 'JAPAN':Varchar) AND (nation.n_name = 'CHINA':Varchar)) OR ((nation.n_name = 'CHINA':Varchar) AND (nation.n_name = 'JAPAN':Varchar))), output: [supplier.s_nationkey, order_line.ol_amount, orders.o_entry_d, customer.c_state], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard($expr2) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, supplier.s_nationkey, nation.n_name, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2] } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, customer.c_state, supplier.s_nationkey, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, customer.c_state, supplier.s_nationkey, nation.n_name], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_amount, orders.o_entry_d, customer.c_state, supplier.s_nationkey] } + └─BatchLookupJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_amount, orders.o_entry_d, customer.c_state, supplier.s_nationkey], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } - └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id AND orders.o_c_id = customer.c_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state] } + └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id AND orders.o_c_id = customer.c_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state], lookup table: customer } └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_c_id, orders.o_d_id, orders.o_w_id) } - └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d] } + └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], lookup table: orders } └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_supply_w_id AND stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(stock.s_i_id, stock.s_w_id) } @@ -1235,19 +1235,19 @@ └─BatchHashAgg { group_key: [$expr3], aggs: [sum($expr4), sum(order_line.ol_amount)] } └─BatchExchange { order: [], dist: HashShard($expr3) } └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_entry_d) as $expr3, Case((nation.n_name = 'INDIA':Varchar), order_line.ol_amount, 0:Decimal) as $expr4, order_line.ol_amount] } - └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'ASIA':Varchar), output: [order_line.ol_amount, orders.o_entry_d, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'ASIA':Varchar), output: [order_line.ol_amount, orders.o_entry_d, nation.n_name], lookup table: region } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } - └─BatchLookupJoin { type: Inner, predicate: $expr2 = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name, nation.n_regionkey] } + └─BatchLookupJoin { type: Inner, predicate: $expr2 = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name, nation.n_regionkey], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard($expr2) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, nation.n_name, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2] } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, customer.c_state, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, customer.c_state, nation.n_name], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_amount, orders.o_entry_d, customer.c_state, supplier.s_nationkey] } + └─BatchLookupJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_amount, orders.o_entry_d, customer.c_state, supplier.s_nationkey], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } - └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id AND orders.o_c_id = customer.c_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state] } + └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id AND orders.o_c_id = customer.c_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state], lookup table: customer } └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_c_id, orders.o_d_id, orders.o_w_id) } - └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d <= '2032-01-02 00:00:00':Timestamp), output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d] } + └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d <= '2032-01-02 00:00:00':Timestamp), output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], lookup table: orders } └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = order_line.ol_supply_w_id AND item.i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(item.i_id, stock.s_w_id) } @@ -1556,14 +1556,14 @@ └─BatchHashAgg { group_key: [nation.n_name, $expr2], aggs: [sum(order_line.ol_amount)] } └─BatchExchange { order: [], dist: HashShard(nation.n_name, $expr2) } └─BatchProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_entry_d) as $expr2, order_line.ol_amount] } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_amount, orders.o_entry_d, supplier.s_nationkey] } + └─BatchLookupJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_amount, orders.o_entry_d, supplier.s_nationkey], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } - └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id, output: [order_line.ol_amount, stock.s_i_id, stock.s_w_id, orders.o_entry_d] } + └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id, output: [order_line.ol_amount, stock.s_i_id, stock.s_w_id, orders.o_entry_d], lookup table: orders } └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - └─BatchLookupJoin { type: Inner, predicate: order_line.ol_supply_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, stock.s_i_id, stock.s_w_id] } + └─BatchLookupJoin { type: Inner, predicate: order_line.ol_supply_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, stock.s_i_id, stock.s_w_id], lookup table: stock } └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } └─BatchHashJoin { type: Inner, predicate: item.i_id = order_line.ol_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(item.i_id) } @@ -1781,7 +1781,7 @@ └─BatchSort { order: [sum(order_line.ol_amount) DESC] } └─BatchHashAgg { group_key: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name], aggs: [sum(order_line.ol_amount)] } └─BatchExchange { order: [], dist: HashShard(customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name) } - └─BatchLookupJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name, order_line.ol_amount] } + └─BatchLookupJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name, order_line.ol_amount], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, order_line.ol_amount, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr1] } └─BatchHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id AND (orders.o_entry_d <= order_line.ol_delivery_d), output: [customer.c_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone, order_line.ol_amount] } @@ -1972,7 +1972,7 @@ │ └─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } │ └─BatchHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt] } │ ├─BatchExchange { order: [], dist: HashShard(supplier.s_suppkey) } - │ │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'CHINA':Varchar), output: [supplier.s_suppkey] } + │ │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'CHINA':Varchar), output: [supplier.s_suppkey], lookup table: nation } │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } │ │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } │ └─BatchExchange { order: [], dist: HashShard($expr1) } @@ -1984,7 +1984,7 @@ └─BatchSimpleAgg { aggs: [sum(stock.s_order_cnt)] } └─BatchHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr3, output: [stock.s_order_cnt] } ├─BatchExchange { order: [], dist: HashShard(supplier.s_suppkey) } - │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'CHINA':Varchar), output: [supplier.s_suppkey] } + │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'CHINA':Varchar), output: [supplier.s_suppkey], lookup table: nation } │ └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } └─BatchExchange { order: [], dist: HashShard($expr3) } @@ -2396,7 +2396,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum($expr1), sum(order_line.ol_amount)] } └─BatchProject { exprs: [Case(Like(item.i_data, 'PR%':Varchar), order_line.ol_amount, 0:Decimal) as $expr1, order_line.ol_amount] } - └─BatchLookupJoin { type: Inner, predicate: order_line.ol_i_id = item.i_id, output: [order_line.ol_amount, item.i_data] } + └─BatchLookupJoin { type: Inner, predicate: order_line.ol_i_id = item.i_id, output: [order_line.ol_amount, item.i_data], lookup table: item } └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id) } └─BatchProject { exprs: [order_line.ol_i_id, order_line.ol_amount] } └─BatchFilter { predicate: (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-02 00:00:00':Timestamp) } @@ -2625,10 +2625,10 @@ └─BatchHashAgg { group_key: [item.i_name, $expr2, item.i_price, $expr3], aggs: [] } └─BatchExchange { order: [], dist: HashShard(item.i_name, $expr2, item.i_price, $expr3) } └─BatchProject { exprs: [item.i_name, Substr(item.i_data, 1:Int32, 3:Int32) as $expr2, item.i_price, ((stock.s_w_id * stock.s_i_id) % 10000:Int32) as $expr3] } - └─BatchLookupJoin { type: LeftAnti, predicate: $expr1 = supplier.s_suppkey AND Like(supplier.s_comment, '%bad%':Varchar), output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data] } + └─BatchLookupJoin { type: LeftAnti, predicate: $expr1 = supplier.s_suppkey AND Like(supplier.s_comment, '%bad%':Varchar), output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } - └─BatchLookupJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND (Not((item.i_data >= 'zz':Varchar)) OR Not((item.i_data < 'z{':Varchar))), output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data] } + └─BatchLookupJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND (Not((item.i_data >= 'zz':Varchar)) OR Not((item.i_data < 'z{':Varchar))), output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data], lookup table: item } └─BatchExchange { order: [], dist: UpstreamHashShard(stock.s_i_id) } └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } stream_plan: |- @@ -3042,7 +3042,7 @@ BatchSimpleAgg { aggs: [sum(sum(order_line.ol_amount))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum(order_line.ol_amount)] } - └─BatchLookupJoin { type: Inner, predicate: order_line.ol_i_id = item.i_id AND (((Like(item.i_data, '%a':Varchar) AND In(order_line.ol_w_id, 1:Int32, 2:Int32, 3:Int32)) OR (Like(item.i_data, '%b':Varchar) AND In(order_line.ol_w_id, 1:Int32, 2:Int32, 4:Int32))) OR (Like(item.i_data, '%c':Varchar) AND In(order_line.ol_w_id, 1:Int32, 5:Int32, 3:Int32))) AND (item.i_price >= 1:Decimal) AND (item.i_price <= 400000:Decimal), output: [order_line.ol_amount] } + └─BatchLookupJoin { type: Inner, predicate: order_line.ol_i_id = item.i_id AND (((Like(item.i_data, '%a':Varchar) AND In(order_line.ol_w_id, 1:Int32, 2:Int32, 3:Int32)) OR (Like(item.i_data, '%b':Varchar) AND In(order_line.ol_w_id, 1:Int32, 2:Int32, 4:Int32))) OR (Like(item.i_data, '%c':Varchar) AND In(order_line.ol_w_id, 1:Int32, 5:Int32, 3:Int32))) AND (item.i_price >= 1:Decimal) AND (item.i_price <= 400000:Decimal), output: [order_line.ol_amount], lookup table: item } └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id) } └─BatchProject { exprs: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount] } └─BatchFilter { predicate: (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 10:Int32) } @@ -3155,14 +3155,14 @@ └─BatchSort { order: [supplier.s_name ASC] } └─BatchHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address] } ├─BatchExchange { order: [], dist: HashShard(supplier.s_suppkey) } - │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'CHINA':Varchar), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address] } + │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'CHINA':Varchar), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address], lookup table: nation } │ └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } └─BatchExchange { order: [], dist: HashShard($expr1) } └─BatchProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32)::Int64 as $expr1] } └─BatchFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) } └─BatchHashAgg { group_key: [stock.s_i_id, stock.s_w_id, stock.s_quantity], aggs: [sum(order_line.ol_quantity)] } - └─BatchLookupJoin { type: LeftSemi, predicate: stock.s_i_id = item.i_id AND (item.i_data >= 'co':Varchar) AND (item.i_data < 'cp':Varchar), output: all } + └─BatchLookupJoin { type: LeftSemi, predicate: stock.s_i_id = item.i_id AND (item.i_data >= 'co':Varchar) AND (item.i_data < 'cp':Varchar), output: all, lookup table: item } └─BatchExchange { order: [], dist: UpstreamHashShard(stock.s_i_id) } └─BatchHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity] } ├─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } @@ -3353,14 +3353,14 @@ └─BatchExchange { order: [], dist: HashShard(supplier.s_name) } └─BatchHashJoin { type: LeftAnti, predicate: order_line.ol_o_id = order_line.ol_o_id AND order_line.ol_w_id = order_line.ol_w_id AND order_line.ol_d_id = order_line.ol_d_id AND (order_line.ol_delivery_d > order_line.ol_delivery_d), output: [supplier.s_name] } ├─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } - │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'CHINA':Varchar), output: [supplier.s_name, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d] } + │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'CHINA':Varchar), output: [supplier.s_name, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d], lookup table: nation } │ └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - │ └─BatchLookupJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, supplier.s_name, supplier.s_nationkey] } + │ └─BatchLookupJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, supplier.s_name, supplier.s_nationkey], lookup table: supplier } │ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } │ └─BatchProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } - │ └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, stock.s_i_id, stock.s_w_id] } + │ └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, stock.s_i_id, stock.s_w_id], lookup table: stock } │ └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id, order_line.ol_w_id) } - │ └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND (order_line.ol_delivery_d > orders.o_entry_d), output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d] } + │ └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND (order_line.ol_delivery_d > orders.o_entry_d), output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d], lookup table: orders } │ └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } │ └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d], distribution: SomeShard } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/distributed_lookup_join.yaml b/src/frontend/planner_test/tests/testdata/output/distributed_lookup_join.yaml index 051988be51a94..a67b123ae18e2 100644 --- a/src/frontend/planner_test/tests/testdata/output/distributed_lookup_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/distributed_lookup_join.yaml @@ -7,7 +7,7 @@ select * from t1 natural join t2; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: t1.a = idx.a AND t1.b = idx.b AND t1.c = idx.c, output: [t1.c, t1.b, t1.a] } + └─BatchLookupJoin { type: Inner, predicate: t1.a = idx.a AND t1.b = idx.b AND t1.c = idx.c, output: [t1.c, t1.b, t1.a], lookup table: idx } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: SomeShard } - id: fix https://github.com/risingwavelabs/risingwave/issues/10721 @@ -18,7 +18,7 @@ select * from t1 natural join t2; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: t1.b = idx.b AND t1.a = idx.a AND t1.c = idx.c, output: [t1.c, t1.b, t1.a] } + └─BatchLookupJoin { type: Inner, predicate: t1.b = idx.b AND t1.a = idx.a AND t1.c = idx.c, output: [t1.c, t1.b, t1.a], lookup table: idx } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.b) } └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: SomeShard } - id: fix https://github.com/risingwavelabs/risingwave/issues/10721 @@ -29,7 +29,7 @@ select * from t1 natural join t2; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: t1.c = idx.c AND t1.b = idx.b AND t1.a = idx.a, output: [t1.c, t1.b, t1.a] } + └─BatchLookupJoin { type: Inner, predicate: t1.c = idx.c AND t1.b = idx.b AND t1.a = idx.a, output: [t1.c, t1.b, t1.a], lookup table: idx } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.c, t1.b) } └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: SomeShard } - id: fix https://github.com/risingwavelabs/risingwave/issues/10721 @@ -40,6 +40,6 @@ select * from t1 natural join t2; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: t1.c = idx.c AND t1.b = idx.b AND t1.a = idx.a, output: [t1.c, t1.b, t1.a] } + └─BatchLookupJoin { type: Inner, predicate: t1.c = idx.c AND t1.b = idx.b AND t1.a = idx.a, output: [t1.c, t1.b, t1.a], lookup table: idx } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.c, t1.b, t1.a) } └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml index 97d77873ff90b..63ee19f4e1470 100644 --- a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml @@ -13,7 +13,7 @@ sql: select A.v, B.v as Bv from A join B using(k1); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: a.k1 = bk1.k1, output: [a.v, bk1.v] } + └─BatchLookupJoin { type: Inner, predicate: a.k1 = bk1.k1, output: [a.v, bk1.v], lookup table: bk1 } └─BatchExchange { order: [], dist: UpstreamHashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1, a.v], distribution: SomeShard } stream_plan: |- @@ -65,7 +65,7 @@ sql: select A.v, B.v as Bv from Ak1 as A join B using(k1) batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v] } + └─BatchLookupJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v], lookup table: bk1 } └─BatchExchange { order: [], dist: UpstreamHashShard(ak1.k1) } └─BatchScan { table: ak1, columns: [ak1.k1, ak1.v], distribution: UpstreamHashShard(ak1.k1) } stream_plan: |- @@ -117,7 +117,7 @@ sql: select A.v, B.v as Bv from A join Bk1 as B using(k1) batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: a.k1 = bk1.k1, output: [a.v, bk1.v] } + └─BatchLookupJoin { type: Inner, predicate: a.k1 = bk1.k1, output: [a.v, bk1.v], lookup table: bk1 } └─BatchExchange { order: [], dist: UpstreamHashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1, a.v], distribution: SomeShard } stream_plan: |- @@ -169,7 +169,7 @@ sql: select A.v, B.v as Bv from Ak1 as A join Bk1 as B using(k1) batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v] } + └─BatchLookupJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v], lookup table: bk1 } └─BatchExchange { order: [], dist: UpstreamHashShard(ak1.k1) } └─BatchScan { table: ak1, columns: [ak1.k1, ak1.v], distribution: UpstreamHashShard(ak1.k1) } stream_plan: |- @@ -1038,7 +1038,7 @@ └─LogicalScan { table: ak1, columns: [ak1.k1, ak1.k2, ak1.k3, ak1.v, ak1.a._row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count] } + └─BatchLookupJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count], lookup table: ak1 } └─BatchExchange { order: [], dist: UpstreamHashShard(a.k1) } └─BatchHashAgg { group_key: [a.k1], aggs: [count] } └─BatchExchange { order: [], dist: HashShard(a.k1) } diff --git a/src/frontend/planner_test/tests/testdata/output/except.yaml b/src/frontend/planner_test/tests/testdata/output/except.yaml index a98c834a19980..6c1b2c498ce4c 100644 --- a/src/frontend/planner_test/tests/testdata/output/except.yaml +++ b/src/frontend/planner_test/tests/testdata/output/except.yaml @@ -115,7 +115,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } - └─BatchLookupJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } + └─BatchLookupJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all, lookup table: t2 } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: UpstreamHashShard(t1.a) } stream_plan: |- diff --git a/src/frontend/planner_test/tests/testdata/output/index_selection.yaml b/src/frontend/planner_test/tests/testdata/output/index_selection.yaml index affc346bda24d..d5a270cdaba77 100644 --- a/src/frontend/planner_test/tests/testdata/output/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/output/index_selection.yaml @@ -58,7 +58,7 @@ select * from t1 where b = 1 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c] } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(1))], distribution: SomeShard } - sql: | @@ -135,11 +135,11 @@ select * from t1 where c = 1 and a < 10; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx3.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.a < 10:Int32), output: [t1.a, t1.b, t1.c] } + └─BatchLookupJoin { type: Inner, predicate: idx3.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.a < 10:Int32), output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx3.t1._row_id) } └─BatchScan { table: idx3, columns: [idx3.t1._row_id], scan_ranges: [idx3.c = Int64(1)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx3.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.a < 10:Int32), output: [t1.a, t1.b, t1.c] } + BatchLookupJoin { type: Inner, predicate: idx3.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.a < 10:Int32), output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx3, columns: [idx3.t1._row_id], scan_ranges: [idx3.c = Int64(1)], distribution: SomeShard } - sql: | @@ -150,11 +150,11 @@ select * from t1 where a = 1; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c] } + └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t1._row_id) } └─BatchScan { table: idx1, columns: [idx1.t1._row_id], scan_ranges: [idx1.a = Int32(1)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c] } + BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx1, columns: [idx1.t1._row_id], scan_ranges: [idx1.a = Int32(1)], distribution: SomeShard } - sql: | @@ -165,11 +165,11 @@ select * from t1 where a = 1 and b = 2; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c] } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c] } + BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } - sql: | @@ -180,11 +180,11 @@ select * from t1 where b = 2; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c] } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2))], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c] } + BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2))], distribution: SomeShard } - sql: | @@ -197,12 +197,12 @@ BatchExchange { order: [], dist: Single } └─BatchDelete { table: t1 } └─BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2))], distribution: SomeShard } batch_local_plan: |- BatchDelete { table: t1 } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id], lookup table: t1 } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2))], distribution: SomeShard } - sql: | @@ -215,12 +215,12 @@ BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } └─BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } batch_local_plan: |- BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id], lookup table: t1 } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } - sql: | @@ -230,7 +230,7 @@ select * from v where cnt = 1 or p = 2; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.v.p IS NOT DISTINCT FROM v.p AND ((v.cnt = 1:Int32) OR (v.p = 2:Int32)), output: [v.cnt, v.p] } + └─BatchLookupJoin { type: Inner, predicate: idx1.v.p IS NOT DISTINCT FROM v.p AND ((v.cnt = 1:Int32) OR (v.p = 2:Int32)), output: [v.cnt, v.p], lookup table: v } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.v.p) } └─BatchHashAgg { group_key: [idx1.v.p], aggs: [] } └─BatchExchange { order: [], dist: HashShard(idx1.v.p) } @@ -240,7 +240,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: v, columns: [v.p], scan_ranges: [v.p = Int32(2)], distribution: UpstreamHashShard(v.p) } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx1.v.p IS NOT DISTINCT FROM v.p AND ((v.cnt = 1:Int32) OR (v.p = 2:Int32)), output: [v.cnt, v.p] } + BatchLookupJoin { type: Inner, predicate: idx1.v.p IS NOT DISTINCT FROM v.p AND ((v.cnt = 1:Int32) OR (v.p = 2:Int32)), output: [v.cnt, v.p], lookup table: v } └─BatchHashAgg { group_key: [idx1.v.p], aggs: [] } └─BatchUnion { all: true } ├─BatchExchange { order: [], dist: Single } @@ -255,7 +255,7 @@ select * from t1 where a = 1 or c = 1 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.a = 1:Int32) OR (t1.c = 1:Int32)), output: [t1.a, t1.b, t1.c] } + └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.a = 1:Int32) OR (t1.c = 1:Int32)), output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t1._row_id) } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchExchange { order: [], dist: HashShard(idx1.t1._row_id) } @@ -265,7 +265,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx3, columns: [idx3.t1._row_id], scan_ranges: [idx3.c = Int64(1)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.a = 1:Int32) OR (t1.c = 1:Int32)), output: [t1.a, t1.b, t1.c] } + BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.a = 1:Int32) OR (t1.c = 1:Int32)), output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchUnion { all: true } ├─BatchExchange { order: [], dist: Single } @@ -280,7 +280,7 @@ select * from t1 where c = 1 or (a = 2 and b = 3) batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.c = 1:Int32) OR ((t1.a = 2:Int32) AND (t1.b = 3:Decimal))), output: [t1.a, t1.b, t1.c] } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.c = 1:Int32) OR ((t1.a = 2:Int32) AND (t1.b = 3:Decimal))), output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchHashAgg { group_key: [idx2.t1._row_id], aggs: [] } └─BatchExchange { order: [], dist: HashShard(idx2.t1._row_id) } @@ -290,7 +290,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx3, columns: [idx3.t1._row_id], scan_ranges: [idx3.c = Int64(1)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.c = 1:Int32) OR ((t1.a = 2:Int32) AND (t1.b = 3:Decimal))), output: [t1.a, t1.b, t1.c] } + BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.c = 1:Int32) OR ((t1.a = 2:Int32) AND (t1.b = 3:Decimal))), output: [t1.a, t1.b, t1.c], lookup table: t1 } └─BatchHashAgg { group_key: [idx2.t1._row_id], aggs: [] } └─BatchUnion { all: true } ├─BatchExchange { order: [], dist: Single } @@ -306,7 +306,7 @@ select * from t1 where p = 1 or (a = 2 and b = 3 and c = 4) batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.p = 1:Int32) OR (((t1.a = 2:Int32) AND (t1.b = 3:Decimal)) AND (t1.c = 4:Int32))), output: [t1.a, t1.b, t1.c, t1.p] } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.p = 1:Int32) OR (((t1.a = 2:Int32) AND (t1.b = 3:Decimal)) AND (t1.c = 4:Int32))), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchHashAgg { group_key: [idx2.t1._row_id], aggs: [] } └─BatchExchange { order: [], dist: HashShard(idx2.t1._row_id) } @@ -316,7 +316,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx4, columns: [idx4.t1._row_id], scan_ranges: [idx4.p = Int32(1)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.p = 1:Int32) OR (((t1.a = 2:Int32) AND (t1.b = 3:Decimal)) AND (t1.c = 4:Int32))), output: [t1.a, t1.b, t1.c, t1.p] } + BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.p = 1:Int32) OR (((t1.a = 2:Int32) AND (t1.b = 3:Decimal)) AND (t1.c = 4:Int32))), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchHashAgg { group_key: [idx2.t1._row_id], aggs: [] } └─BatchUnion { all: true } ├─BatchExchange { order: [], dist: Single } @@ -332,7 +332,7 @@ select * from t1 where a = 1 or b = 2 or c = 3 or p = 4 or a = 5 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((((t1.a = 1:Int32) OR (t1.b = 2:Decimal)) OR (t1.c = 3:Int32)) OR (t1.p = 4:Int32)) OR (t1.a = 5:Int32)), output: [t1.a, t1.b, t1.c, t1.p] } + └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((((t1.a = 1:Int32) OR (t1.b = 2:Decimal)) OR (t1.c = 3:Int32)) OR (t1.p = 4:Int32)) OR (t1.a = 5:Int32)), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t1._row_id) } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchExchange { order: [], dist: HashShard(idx1.t1._row_id) } @@ -346,7 +346,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx4, columns: [idx4.t1._row_id], scan_ranges: [idx4.p = Int32(4)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((((t1.a = 1:Int32) OR (t1.b = 2:Decimal)) OR (t1.c = 3:Int32)) OR (t1.p = 4:Int32)) OR (t1.a = 5:Int32)), output: [t1.a, t1.b, t1.c, t1.p] } + BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((((t1.a = 1:Int32) OR (t1.b = 2:Decimal)) OR (t1.c = 3:Int32)) OR (t1.p = 4:Int32)) OR (t1.a = 5:Int32)), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchUnion { all: true } ├─BatchExchange { order: [], dist: Single } @@ -366,7 +366,7 @@ select * from t1 where (a = 1 or (b = 2 and a = 5)) and (c = 3 or p = 4) batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.a = 1:Int32) OR ((t1.b = 2:Decimal) AND (t1.a = 5:Int32))) AND ((t1.c = 3:Int32) OR (t1.p = 4:Int32)), output: [t1.a, t1.b, t1.c, t1.p] } + └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.a = 1:Int32) OR ((t1.b = 2:Decimal) AND (t1.a = 5:Int32))) AND ((t1.c = 3:Int32) OR (t1.p = 4:Int32)), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t1._row_id) } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchExchange { order: [], dist: HashShard(idx1.t1._row_id) } @@ -376,7 +376,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(5)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.a = 1:Int32) OR ((t1.b = 2:Decimal) AND (t1.a = 5:Int32))) AND ((t1.c = 3:Int32) OR (t1.p = 4:Int32)), output: [t1.a, t1.b, t1.c, t1.p] } + BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((t1.a = 1:Int32) OR ((t1.b = 2:Decimal) AND (t1.a = 5:Int32))) AND ((t1.c = 3:Int32) OR (t1.p = 4:Int32)), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchUnion { all: true } ├─BatchExchange { order: [], dist: Single } @@ -392,7 +392,7 @@ select * from t1 where p != 1 and (c = 3 or (c != 4 and (a = 2 or b = 3))) batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.p <> 1:Int32) AND ((t1.c = 3:Int32) OR ((t1.c <> 4:Int32) AND ((t1.a = 2:Int32) OR (t1.b = 3:Decimal)))), output: [t1.a, t1.b, t1.c, t1.p] } + └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.p <> 1:Int32) AND ((t1.c = 3:Int32) OR ((t1.c <> 4:Int32) AND ((t1.a = 2:Int32) OR (t1.b = 3:Decimal)))), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t1._row_id) } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchExchange { order: [], dist: HashShard(idx1.t1._row_id) } @@ -404,7 +404,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx3, columns: [idx3.t1._row_id], scan_ranges: [idx3.c = Int64(3)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.p <> 1:Int32) AND ((t1.c = 3:Int32) OR ((t1.c <> 4:Int32) AND ((t1.a = 2:Int32) OR (t1.b = 3:Decimal)))), output: [t1.a, t1.b, t1.c, t1.p] } + BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.p <> 1:Int32) AND ((t1.c = 3:Int32) OR ((t1.c <> 4:Int32) AND ((t1.a = 2:Int32) OR (t1.b = 3:Decimal)))), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchUnion { all: true } ├─BatchExchange { order: [], dist: Single } @@ -423,7 +423,7 @@ select * from t1 where (a > 1 and a < 8) or c between 8 and 9 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((t1.a > 1:Int32) AND (t1.a < 8:Int32)) OR ((t1.c >= 8:Int32) AND (t1.c <= 9:Int32))), output: [t1.a, t1.b, t1.c, t1.p] } + └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((t1.a > 1:Int32) AND (t1.a < 8:Int32)) OR ((t1.c >= 8:Int32) AND (t1.c <= 9:Int32))), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t1._row_id) } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchExchange { order: [], dist: HashShard(idx1.t1._row_id) } @@ -433,7 +433,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx3, columns: [idx3.t1._row_id], scan_ranges: [idx3.c = Int64(8), idx3.c = Int64(9)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((t1.a > 1:Int32) AND (t1.a < 8:Int32)) OR ((t1.c >= 8:Int32) AND (t1.c <= 9:Int32))), output: [t1.a, t1.b, t1.c, t1.p] } + BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((t1.a > 1:Int32) AND (t1.a < 8:Int32)) OR ((t1.c >= 8:Int32) AND (t1.c <= 9:Int32))), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchUnion { all: true } ├─BatchExchange { order: [], dist: Single } @@ -449,7 +449,7 @@ select * from t1 where (a > 1 and a < 8) or c = 8 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((t1.a > 1:Int32) AND (t1.a < 8:Int32)) OR (t1.c = 8:Int32)), output: [t1.a, t1.b, t1.c, t1.p] } + └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((t1.a > 1:Int32) AND (t1.a < 8:Int32)) OR (t1.c = 8:Int32)), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t1._row_id) } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchExchange { order: [], dist: HashShard(idx1.t1._row_id) } @@ -459,7 +459,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx3, columns: [idx3.t1._row_id], scan_ranges: [idx3.c = Int64(8)], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((t1.a > 1:Int32) AND (t1.a < 8:Int32)) OR (t1.c = 8:Int32)), output: [t1.a, t1.b, t1.c, t1.p] } + BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((t1.a > 1:Int32) AND (t1.a < 8:Int32)) OR (t1.c = 8:Int32)), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] } └─BatchUnion { all: true } ├─BatchExchange { order: [], dist: Single } @@ -547,11 +547,11 @@ select * from t1 where a in ( 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, 98, 99,100); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1.p] } + └─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t1._row_id) } └─BatchScan { table: idx1, columns: [idx1.t1._row_id], scan_ranges: [idx1.a = Int32(1), idx1.a = Int32(2), idx1.a = Int32(3), idx1.a = Int32(4), idx1.a = Int32(5), idx1.a = Int32(6), idx1.a = Int32(7), idx1.a = Int32(8), idx1.a = Int32(9), idx1.a = Int32(10), idx1.a = Int32(11), idx1.a = Int32(12), idx1.a = Int32(13), idx1.a = Int32(14), idx1.a = Int32(15), idx1.a = Int32(16), idx1.a = Int32(17), idx1.a = Int32(18), idx1.a = Int32(19), idx1.a = Int32(20), ...], distribution: SomeShard } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1.p] } + BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx1, columns: [idx1.t1._row_id], scan_ranges: [idx1.a = Int32(1), idx1.a = Int32(2), idx1.a = Int32(3), idx1.a = Int32(4), idx1.a = Int32(5), idx1.a = Int32(6), idx1.a = Int32(7), idx1.a = Int32(8), idx1.a = Int32(9), idx1.a = Int32(10), idx1.a = Int32(11), idx1.a = Int32(12), idx1.a = Int32(13), idx1.a = Int32(14), idx1.a = Int32(15), idx1.a = Int32(16), idx1.a = Int32(17), idx1.a = Int32(18), idx1.a = Int32(19), idx1.a = Int32(20), ...], distribution: SomeShard } - sql: | @@ -562,12 +562,12 @@ BatchSimpleAgg { aggs: [sum0(count)] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [count] } - └─BatchLookupJoin { type: Inner, predicate: d.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.d > 9:Int32), output: [] } + └─BatchLookupJoin { type: Inner, predicate: d.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.d > 9:Int32), output: [], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(d.t1._row_id) } └─BatchScan { table: d, columns: [d.t1._row_id], scan_ranges: [d.a = Int32(1)], distribution: SomeShard } batch_local_plan: |- BatchSimpleAgg { aggs: [count] } - └─BatchLookupJoin { type: Inner, predicate: d.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.d > 9:Int32), output: [] } + └─BatchLookupJoin { type: Inner, predicate: d.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (t1.d > 9:Int32), output: [], lookup table: t1 } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: d, columns: [d.t1._row_id], scan_ranges: [d.a = Int32(1)], distribution: SomeShard } - name: create index to include all columns by default @@ -678,7 +678,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [(t1.v1 + 1:Int32) as $expr1, (t2.v2 + 2:Int32) as $expr2] } - └─BatchLookupJoin { type: Inner, predicate: t1.k1 = t2.k2, output: all } + └─BatchLookupJoin { type: Inner, predicate: t1.k1 = t2.k2, output: all, lookup table: t2 } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.k1) } └─BatchScan { table: t1, columns: [t1.k1, t1.v1], distribution: UpstreamHashShard(t1.k1) } - sql: | @@ -690,7 +690,7 @@ BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t2.d1, t2.d2, idx_t1.c1, idx_t1.c2, idx_t1.c3, $expr1, ($expr1 + '00:10:00':Interval) as $expr2] } └─BatchProject { exprs: [t2.d1, t2.d2, idx_t1.c1, idx_t1.c2, idx_t1.c3, TumbleStart(idx_t1.c3, '00:10:00':Interval) as $expr1] } - └─BatchLookupJoin { type: Inner, predicate: t2.d1 = idx_t1.c2, output: all } + └─BatchLookupJoin { type: Inner, predicate: t2.d1 = idx_t1.c2, output: all, lookup table: idx_t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(t2.d1) } └─BatchScan { table: t2, columns: [t2.d1, t2.d2], distribution: SomeShard } - sql: | @@ -706,7 +706,7 @@ select * from t where j->>'k1' = 'abc'; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.t._row_id IS NOT DISTINCT FROM t._row_id, output: [t.j, t.v1, t.v2] } + └─BatchLookupJoin { type: Inner, predicate: idx1.t._row_id IS NOT DISTINCT FROM t._row_id, output: [t.j, t.v1, t.v2], lookup table: t } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t._row_id) } └─BatchScan { table: idx1, columns: [idx1.t._row_id], scan_ranges: [idx1.JSONB_ACCESS_STR = Utf8("abc")], distribution: SomeShard } - sql: | @@ -716,7 +716,7 @@ select * from t where j->>'k1' = 'abc' or j->>'k2' = 'ABC'; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx1.t._row_id IS NOT DISTINCT FROM t._row_id AND ((JsonbAccessStr(t.j, 'k1':Varchar) = 'abc':Varchar) OR (JsonbAccessStr(t.j, 'k2':Varchar) = 'ABC':Varchar)), output: [t.j, t.v1, t.v2] } + └─BatchLookupJoin { type: Inner, predicate: idx1.t._row_id IS NOT DISTINCT FROM t._row_id AND ((JsonbAccessStr(t.j, 'k1':Varchar) = 'abc':Varchar) OR (JsonbAccessStr(t.j, 'k2':Varchar) = 'ABC':Varchar)), output: [t.j, t.v1, t.v2], lookup table: t } └─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t._row_id) } └─BatchHashAgg { group_key: [idx1.t._row_id], aggs: [] } └─BatchExchange { order: [], dist: HashShard(idx1.t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/intersect.yaml b/src/frontend/planner_test/tests/testdata/output/intersect.yaml index 5badfae8c9f73..81bb53b26ffb2 100644 --- a/src/frontend/planner_test/tests/testdata/output/intersect.yaml +++ b/src/frontend/planner_test/tests/testdata/output/intersect.yaml @@ -115,7 +115,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } - └─BatchLookupJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } + └─BatchLookupJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all, lookup table: t2 } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: UpstreamHashShard(t1.a) } stream_plan: |- diff --git a/src/frontend/planner_test/tests/testdata/output/join.yaml b/src/frontend/planner_test/tests/testdata/output/join.yaml index 387862e92861e..2ae639511f20b 100644 --- a/src/frontend/planner_test/tests/testdata/output/join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join.yaml @@ -154,7 +154,7 @@ select i.x as ix, ii.x as iix from i join i as ii on i.x=ii.x; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: all } + └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: all, lookup table: i } └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: |- StreamMaterialize { columns: [ix, iix, i.t._row_id(hidden), i.t._row_id#1(hidden)], stream_key: [i.t._row_id, i.t._row_id#1, ix], pk_columns: [i.t._row_id, i.t._row_id#1, ix], pk_conflict: NoCheck } @@ -171,7 +171,7 @@ select i.x as ix, t.x as tx from i join t on i.x=t.x; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: all } + └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: all, lookup table: i } └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: |- StreamMaterialize { columns: [ix, tx, i.t._row_id(hidden), t._row_id(hidden)], stream_key: [i.t._row_id, t._row_id, ix], pk_columns: [i.t._row_id, t._row_id, ix], pk_conflict: NoCheck } @@ -195,10 +195,10 @@ └─BatchProject { exprs: [Coalesce(i.x, i.x) as $expr1] } └─BatchHashJoin { type: FullOuter, predicate: i.x = i.x, output: all } ├─BatchExchange { order: [], dist: HashShard(i.x) } - │ └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: [i.x] } + │ └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: [i.x], lookup table: i } │ └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } └─BatchExchange { order: [], dist: HashShard(i.x) } - └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: [i.x] } + └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: [i.x], lookup table: i } └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: |- StreamMaterialize { columns: [x, i.t._row_id(hidden), i.t._row_id#1(hidden), i.x(hidden), i.t._row_id#2(hidden), i.t._row_id#3(hidden), i.x#1(hidden)], stream_key: [i.t._row_id, i.t._row_id#1, i.x, i.t._row_id#2, i.t._row_id#3, i.x#1], pk_columns: [i.t._row_id, i.t._row_id#1, i.x, i.t._row_id#2, i.t._row_id#3, i.x#1], pk_conflict: NoCheck } @@ -225,7 +225,7 @@ create materialized view t3 as select v1, count(v2) as v2 from t2 group by v1; select * from t1 cross join t3 where t1.v2 = t3.v1; batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: t1.v2 = t3.v1, output: all } + BatchLookupJoin { type: Inner, predicate: t1.v2 = t3.v1, output: all, lookup table: t3 } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t1, columns: [t1.v1, t1.v2], distribution: SomeShard } with_config_map: @@ -578,7 +578,7 @@ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: (t1.v2 > 1:Int32) } └─LogicalScan { table: t3, columns: [t3.v1, t3.v2], predicate: (t3.v1 > 1:Int32) } batch_local_plan: |- - BatchLookupJoin { type: Inner, predicate: t1.v2 = t3.v1 AND (t3.v1 > 1:Int32), output: all } + BatchLookupJoin { type: Inner, predicate: t1.v2 = t3.v1 AND (t3.v1 > 1:Int32), output: all, lookup table: t3 } └─BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (t1.v2 > 1:Int32) } └─BatchScan { table: t1, columns: [t1.v1, t1.v2], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index cd3fb179a43f7..dad8d469dcba9 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -174,7 +174,7 @@ A.category = 10 and (P.state = 'or' OR P.state = 'id' OR P.state = 'ca'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: auction.seller = person.id AND (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)), output: [person.name, person.city, person.state, auction.id] } + └─BatchLookupJoin { type: Inner, predicate: auction.seller = person.id AND (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)), output: [person.name, person.city, person.state, auction.id], lookup table: person } └─BatchExchange { order: [], dist: UpstreamHashShard(auction.seller) } └─BatchProject { exprs: [auction.id, auction.seller] } └─BatchFilter { predicate: (auction.category = 10:Int32) } @@ -1798,7 +1798,7 @@ WHERE A.category = 10; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: bid.auction = auction.id AND (auction.category = 10:Int32), output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category] } + └─BatchLookupJoin { type: Inner, predicate: bid.auction = auction.id AND (auction.category = 10:Int32), output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], lookup table: auction } └─BatchExchange { order: [], dist: UpstreamHashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time], distribution: SomeShard } stream_plan: |- diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 309892e692b02..812471bc5ec2b 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -847,7 +847,7 @@ BatchHopWindow { time_col: t1.ts, slide: 00:10:00, size: 00:30:00, output: all } └─BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: IsNotNull(t1.ts) } - └─BatchLookupJoin { type: Inner, predicate: 1:Int32 = t1.k AND IsNotNull(t1.ts), output: all } + └─BatchLookupJoin { type: Inner, predicate: 1:Int32 = t1.k AND IsNotNull(t1.ts), output: all, lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(1:Int32) } └─BatchHashAgg { group_key: [1:Int32], aggs: [] } └─BatchExchange { order: [], dist: HashShard(1:Int32) } diff --git a/src/frontend/planner_test/tests/testdata/output/time_window.yaml b/src/frontend/planner_test/tests/testdata/output/time_window.yaml index ea7d9c5fbe4d5..de32fd358aa68 100644 --- a/src/frontend/planner_test/tests/testdata/output/time_window.yaml +++ b/src/frontend/planner_test/tests/testdata/output/time_window.yaml @@ -237,7 +237,7 @@ BatchHopWindow { time_col: idx_t1.c3, slide: 00:10:00, size: 00:20:00, output: all } └─BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: IsNotNull(idx_t1.c3) } - └─BatchLookupJoin { type: Inner, predicate: t2.d1 = idx_t1.c2 AND IsNotNull(idx_t1.c3), output: all } + └─BatchLookupJoin { type: Inner, predicate: t2.d1 = idx_t1.c2 AND IsNotNull(idx_t1.c3), output: all, lookup table: idx_t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(t2.d1) } └─BatchScan { table: t2, columns: [t2.d1, t2.d2, t2.d3], distribution: SomeShard } - sql: | @@ -268,7 +268,7 @@ └─BatchFilter { predicate: IsNotNull(idx_t1.c3) } └─BatchHopWindow { time_col: t2.d3, slide: 00:10:00, size: 00:20:00, output: all } └─BatchFilter { predicate: IsNotNull(t2.d3) } - └─BatchLookupJoin { type: Inner, predicate: t2.d1 = idx_t1.c2 AND IsNotNull(idx_t1.c3), output: all } + └─BatchLookupJoin { type: Inner, predicate: t2.d1 = idx_t1.c2 AND IsNotNull(idx_t1.c3), output: all, lookup table: idx_t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(t2.d1) } └─BatchFilter { predicate: IsNotNull(t2.d3) } └─BatchScan { table: t2, columns: [t2.d1, t2.d2, t2.d3], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/output/tpch.yaml b/src/frontend/planner_test/tests/testdata/output/tpch.yaml index 1037300ef9669..69ebe2213cb79 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch.yaml @@ -289,13 +289,13 @@ BatchTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } └─BatchExchange { order: [], dist: Single } └─BatchTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } - └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'AFRICA':Varchar), output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment] } + └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'AFRICA':Varchar), output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment], lookup table: region } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } └─BatchHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND partsupp.ps_supplycost = min(partsupp.ps_supplycost), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey] } ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } - │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey] } + │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey], lookup table: nation } │ └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - │ └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } + │ └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], lookup table: supplier } │ └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } │ └─BatchHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost] } │ ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } @@ -312,11 +312,11 @@ │ └─BatchFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } │ └─BatchScan { table: part, columns: [part.p_partkey, part.p_type, part.p_size], distribution: UpstreamHashShard(part.p_partkey) } └─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey) } - └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'AFRICA':Varchar), output: [partsupp.ps_partkey, partsupp.ps_supplycost] } + └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'AFRICA':Varchar), output: [partsupp.ps_partkey, partsupp.ps_supplycost], lookup table: region } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey] } + └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } └─BatchFilter { predicate: IsNotNull(partsupp.ps_partkey) } └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } @@ -971,9 +971,9 @@ └─BatchHashAgg { group_key: [nation.n_name], aggs: [sum($expr1)] } └─BatchExchange { order: [], dist: HashShard(nation.n_name) } └─BatchProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1] } - └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'MIDDLE EAST':Varchar), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'MIDDLE EAST':Varchar), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name], lookup table: region } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } └─BatchHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey, supplier.s_suppkey) } @@ -1326,13 +1326,13 @@ └─BatchHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2)] } └─BatchExchange { order: [], dist: HashShard(nation.n_name, nation.n_name, $expr1) } └─BatchProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2] } - └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey AND (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))), output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey AND (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))), output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, nation.n_name], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(customer.c_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, customer.c_nationkey] } + └─BatchLookupJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, customer.c_nationkey], lookup table: customer } └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_custkey) } - └─BatchLookupJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, orders.o_custkey] } + └─BatchLookupJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, orders.o_custkey], lookup table: orders } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_orderkey) } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } └─BatchHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate] } ├─BatchExchange { order: [], dist: HashShard(supplier.s_suppkey) } @@ -1612,17 +1612,17 @@ └─BatchExchange { order: [], dist: HashShard($expr1) } └─BatchProject { exprs: [$expr1, Case((nation.n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2] } └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_name] } - └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'ASIA':Varchar), output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'ASIA':Varchar), output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name], lookup table: region } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } - └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey] } + └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(customer.c_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, customer.c_nationkey] } + └─BatchLookupJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, customer.c_nationkey], lookup table: customer } └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_custkey) } - └─BatchLookupJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey AND (orders.o_orderdate >= '1995-01-01':Date) AND (orders.o_orderdate <= '1996-12-31':Date), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_custkey, orders.o_orderdate] } + └─BatchLookupJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey AND (orders.o_orderdate >= '1995-01-01':Date) AND (orders.o_orderdate <= '1996-12-31':Date), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_custkey, orders.o_orderdate], lookup table: orders } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_orderkey) } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey] } + └─BatchLookupJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_suppkey) } └─BatchHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount] } ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } @@ -1945,13 +1945,13 @@ └─BatchHashAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2)] } └─BatchExchange { order: [], dist: HashShard(nation.n_name, $expr1) } └─BatchProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2] } - └─BatchLookupJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name], lookup table: orders } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_orderkey) } - └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = partsupp.ps_partkey AND lineitem.l_suppkey = partsupp.ps_suppkey, output: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, partsupp.ps_supplycost] } + └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = partsupp.ps_partkey AND lineitem.l_suppkey = partsupp.ps_suppkey, output: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, partsupp.ps_supplycost], lookup table: partsupp } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey, lineitem.l_suppkey) } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey] } + └─BatchLookupJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_suppkey) } └─BatchHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount] } ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } @@ -2203,7 +2203,7 @@ └─BatchProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1] } └─BatchHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } - │ └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name] } + │ └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name], lookup table: nation } │ └─BatchExchange { order: [], dist: UpstreamHashShard(customer.c_nationkey) } │ └─BatchHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey] } │ ├─BatchExchange { order: [], dist: HashShard(customer.c_custkey) } @@ -2428,9 +2428,9 @@ │ └─BatchHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1)] } │ └─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey) } │ └─BatchProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr1] } - │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'ARGENTINA':Varchar), output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost] } + │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'ARGENTINA':Varchar), output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost], lookup table: nation } │ └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - │ └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey] } + │ └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey], lookup table: supplier } │ └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } │ └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─BatchProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } @@ -2438,9 +2438,9 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum($expr2)] } └─BatchProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2] } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'ARGENTINA':Varchar), output: [partsupp.ps_availqty, partsupp.ps_supplycost] } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'ARGENTINA':Varchar), output: [partsupp.ps_availqty, partsupp.ps_supplycost], lookup table: nation } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey] } + └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } └─BatchScan { table: partsupp, columns: [partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], distribution: SomeShard } stream_plan: |- @@ -2947,7 +2947,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum($expr1), sum($expr2)] } └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2] } - └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type] } + └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type], lookup table: part } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount] } └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } @@ -3276,9 +3276,9 @@ └─BatchHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(partsupp.ps_suppkey)] } └─BatchExchange { order: [], dist: HashShard(part.p_brand, part.p_type, part.p_size) } └─BatchHashAgg { group_key: [part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey], aggs: [] } - └─BatchLookupJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey AND Like(supplier.s_comment, '%Customer%Complaints%':Varchar), output: [part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey] } + └─BatchLookupJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey AND Like(supplier.s_comment, '%Customer%Complaints%':Varchar), output: [part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey], lookup table: supplier } └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } - └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_partkey = part.p_partkey AND (part.p_brand <> 'Brand#45':Varchar) AND (Not((part.p_type >= 'SMALL PLATED':Varchar)) OR Not((part.p_type < 'SMALL PLATEE':Varchar))) AND In(part.p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32), output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size] } + └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_partkey = part.p_partkey AND (part.p_brand <> 'Brand#45':Varchar) AND (Not((part.p_type >= 'SMALL PLATED':Varchar)) OR Not((part.p_type < 'SMALL PLATEE':Varchar))) AND In(part.p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32), output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size], lookup table: part } └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_partkey) } └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } stream_plan: |- @@ -3483,7 +3483,7 @@ └─BatchSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } └─BatchHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND (lineitem.l_quantity < $expr1), output: [lineitem.l_extendedprice] } ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } - │ └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey AND (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar), output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey] } + │ └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey AND (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar), output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey], lookup table: part } │ └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } │ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice], distribution: SomeShard } └─BatchProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } @@ -3927,7 +3927,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum($expr1)] } └─BatchProject { exprs: [(lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1] } - └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey AND (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Decimal)) AND (lineitem.l_quantity <= 11:Decimal)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Decimal)) AND (lineitem.l_quantity <= 40:Decimal)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Decimal)) AND (lineitem.l_quantity <= 20:Decimal)) AND (part.p_size <= 15:Int32))) AND (part.p_size >= 1:Int32), output: [lineitem.l_extendedprice, lineitem.l_discount] } + └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey AND (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Decimal)) AND (lineitem.l_quantity <= 11:Decimal)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Decimal)) AND (lineitem.l_quantity <= 40:Decimal)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Decimal)) AND (lineitem.l_quantity <= 20:Decimal)) AND (part.p_size <= 15:Int32))) AND (part.p_size >= 1:Int32), output: [lineitem.l_extendedprice, lineitem.l_discount], lookup table: part } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount] } └─BatchFilter { predicate: In(lineitem.l_shipmode, 'AIR':Varchar, 'AIR REG':Varchar) AND (lineitem.l_shipinstruct = 'DELIVER IN PERSON':Varchar) } @@ -4041,11 +4041,11 @@ └─BatchTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } └─BatchHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address] } ├─BatchExchange { order: [], dist: HashShard(supplier.s_suppkey) } - │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'KENYA':Varchar), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address] } + │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'KENYA':Varchar), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address], lookup table: nation } │ └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } └─BatchExchange { order: [], dist: HashShard(partsupp.ps_suppkey) } - └─BatchLookupJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey AND (part.p_name >= 'forest':Varchar) AND (part.p_name < 'foresu':Varchar), output: [partsupp.ps_suppkey] } + └─BatchLookupJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey AND (part.p_name >= 'forest':Varchar) AND (part.p_name < 'foresu':Varchar), output: [partsupp.ps_suppkey], lookup table: part } └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_partkey) } └─BatchHashJoin { type: Inner, predicate: partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey AND ($expr1 > $expr2), output: [partsupp.ps_partkey, partsupp.ps_suppkey] } ├─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } @@ -4314,9 +4314,9 @@ └─BatchHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name] } ├─BatchHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } │ ├─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } - │ │ └─BatchLookupJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey AND (orders.o_orderstatus = 'F':Varchar), output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey] } + │ │ └─BatchLookupJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey AND (orders.o_orderstatus = 'F':Varchar), output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey], lookup table: orders } │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_orderkey) } - │ │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'GERMANY':Varchar), output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey] } + │ │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'GERMANY':Varchar), output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey], lookup table: nation } │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } │ │ └─BatchHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_name, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_suppkey] } │ │ ├─BatchExchange { order: [], dist: HashShard(supplier.s_suppkey) } diff --git a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs index b78bf314c1276..2815ad70e08d2 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -25,7 +25,7 @@ use super::ExprRewritable; use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::{ - EqJoinPredicate, EqJoinPredicateDisplay, PlanBase, PlanTreeNodeUnary, ToBatchPb, + EqJoinPredicate, EqJoinPredicateDisplay, LogicalScan, PlanBase, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, ToLocalBatch, }; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -131,6 +131,11 @@ impl Distill for BatchLookupJoin { vec.push(("output", data)); } + if let Some(scan) = self.core.right.as_logical_scan() { + let scan: &LogicalScan = scan; + vec.push(("lookup table", Pretty::display(&scan.table_name()))); + } + childless_record("BatchLookupJoin", vec) } } From 4e4e8b5b6bf2502b275e15ae94cd479d74625d4c Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Wed, 8 Nov 2023 19:37:40 +0800 Subject: [PATCH 12/77] fix: ignore unknow field in config (#13320) Co-authored-by: ZENOTME --- src/connector/src/sink/iceberg.rs | 2 ++ src/connector/with_options.yaml | 3 +++ 2 files changed, 5 insertions(+) diff --git a/src/connector/src/sink/iceberg.rs b/src/connector/src/sink/iceberg.rs index 86a3c21b83971..b7cf899f443dc 100644 --- a/src/connector/src/sink/iceberg.rs +++ b/src/connector/src/sink/iceberg.rs @@ -66,6 +66,8 @@ pub type RemoteIcebergSink = CoordinatedRemoteSink; #[derive(Debug, Clone, Deserialize, WithOptions)] #[serde(deny_unknown_fields)] pub struct IcebergConfig { + pub connector: String, // Avoid deny unknown field. Must be "iceberg" + pub r#type: String, // accept "append-only" or "upsert" #[serde(default, deserialize_with = "deserialize_bool_from_string")] diff --git a/src/connector/with_options.yaml b/src/connector/with_options.yaml index 63eedfb019ed9..b5abc5512ff03 100644 --- a/src/connector/with_options.yaml +++ b/src/connector/with_options.yaml @@ -66,6 +66,9 @@ DorisConfig: required: true IcebergConfig: fields: + - name: connector + field_type: String + required: true - name: r#type field_type: String required: true From 54390d3fe8db422f796ab8878b6dd50ed168562e Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Wed, 8 Nov 2023 20:50:36 +0800 Subject: [PATCH 13/77] feat: add `properties.fetch.queue.backoff.ms` for kafka source (#13321) --- src/connector/src/source/kafka/mod.rs | 18 +++++++++++------- src/connector/with_options.yaml | 6 +++++- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/connector/src/source/kafka/mod.rs b/src/connector/src/source/kafka/mod.rs index 78c12ea31d343..aa6c3dfb8b147 100644 --- a/src/connector/src/source/kafka/mod.rs +++ b/src/connector/src/source/kafka/mod.rs @@ -63,10 +63,9 @@ pub struct RdKafkaPropertiesConsumer { /// exceeded. This property may need to be decreased if the queue thresholds are set low /// and the application is experiencing long (~1s) delays between messages. Low values may /// increase CPU utilization. - // FIXME: need to upgrade rdkafka to v2.2.0 to use this property - // #[serde(rename = "properties.fetch.queue.backoff.ms")] - // #[serde_as(as = "Option")] - // pub fetch_queue_backoff_ms: Option, + #[serde(rename = "properties.fetch.queue.backoff.ms")] + #[serde_as(as = "Option")] + pub fetch_queue_backoff_ms: Option, /// Maximum amount of data the broker shall return for a Fetch request. Messages are fetched in /// batches by the consumer and if the first message batch in the first non-empty partition of @@ -156,9 +155,9 @@ impl RdKafkaPropertiesConsumer { if let Some(v) = &self.fetch_wait_max_ms { c.set("fetch.wait.max.ms", v.to_string()); } - // if let Some(v) = &self.fetch_queue_backoff_ms { - // c.set("fetch.queue.backoff.ms", v.to_string()); - // } + if let Some(v) = &self.fetch_queue_backoff_ms { + c.set("fetch.queue.backoff.ms", v.to_string()); + } if let Some(v) = &self.fetch_max_bytes { c.set("fetch.max.bytes", v.to_string()); } @@ -193,6 +192,7 @@ mod test { "properties.fetch.wait.max.ms".to_string() => "114514".to_string(), "properties.fetch.max.bytes".to_string() => "114514".to_string(), "properties.enable.auto.commit".to_string() => "true".to_string(), + "properties.fetch.queue.backoff.ms".to_string() => "114514".to_string(), }; let props: KafkaProperties = @@ -215,5 +215,9 @@ mod test { assert_eq!(props.rdkafka_properties.fetch_wait_max_ms, Some(114514)); assert_eq!(props.rdkafka_properties.fetch_max_bytes, Some(114514)); assert_eq!(props.rdkafka_properties.enable_auto_commit, Some(true)); + assert_eq!( + props.rdkafka_properties.fetch_queue_backoff_ms, + Some(114514) + ); } } diff --git a/src/connector/with_options.yaml b/src/connector/with_options.yaml index b5abc5512ff03..d93c90c4aecb3 100644 --- a/src/connector/with_options.yaml +++ b/src/connector/with_options.yaml @@ -382,9 +382,13 @@ KafkaProperties: field_type: Option < usize > comments: Maximum time the broker may wait to fill the Fetch response with `fetch.min.`bytes of messages. required: false + - name: properties.fetch.queue.backoff.ms + field_type: Option < usize > + comments: How long to postpone the next fetch request for a topic+partition in case the current fetch queue thresholds (`queued.min.messages` or `queued.max.messages.kbytes`) have been exceeded. This property may need to be decreased if the queue thresholds are set low and the application is experiencing long (~1s) delays between messages. Low values may increase CPU utilization. + required: false - name: properties.fetch.max.bytes field_type: Option < usize > - comments: How long to postpone the next fetch request for a topic+partition in case the current fetch queue thresholds (`queued.min.messages` or `queued.max.messages.kbytes`) have been exceeded. This property may need to be decreased if the queue thresholds are set low and the application is experiencing long (~1s) delays between messages. Low values may increase CPU utilization. Maximum amount of data the broker shall return for a Fetch request. Messages are fetched in batches by the consumer and if the first message batch in the first non-empty partition of the Fetch request is larger than this value, then the message batch will still be returned to ensure the consumer can make progress. The maximum message batch size accepted by the broker is defined via `message.max.bytes` (broker config) or `max.message.bytes` (broker topic config). `fetch.max.bytes` is automatically adjusted upwards to be at least `message.max.bytes` (consumer config). + comments: Maximum amount of data the broker shall return for a Fetch request. Messages are fetched in batches by the consumer and if the first message batch in the first non-empty partition of the Fetch request is larger than this value, then the message batch will still be returned to ensure the consumer can make progress. The maximum message batch size accepted by the broker is defined via `message.max.bytes` (broker config) or `max.message.bytes` (broker topic config). `fetch.max.bytes` is automatically adjusted upwards to be at least `message.max.bytes` (consumer config). required: false - name: properties.enable.auto.commit field_type: Option < bool > From 7b3f8fca82109465996c57a5d242d6bc9e187b63 Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 8 Nov 2023 21:41:20 +0800 Subject: [PATCH 14/77] feat(frontend): support show processlist (#13287) --- src/frontend/planner_test/src/lib.rs | 19 ++-- .../src/handler/alter_table_column.rs | 4 +- src/frontend/src/handler/extended_handle.rs | 28 +++--- src/frontend/src/handler/mod.rs | 9 +- src/frontend/src/handler/show.rs | 28 ++++++ .../src/optimizer/optimizer_context.rs | 4 +- src/frontend/src/session.rs | 99 +++++++++++++++---- src/frontend/src/test_utils.rs | 20 ++-- src/frontend/src/utils/infer_stmt_row_desc.rs | 32 ++++++ src/sqlparser/src/ast/mod.rs | 2 + src/sqlparser/src/keywords.rs | 1 + src/sqlparser/src/parser.rs | 6 ++ src/tests/sqlsmith/tests/frontend/mod.rs | 20 ++-- src/utils/pgwire/src/net.rs | 5 +- src/utils/pgwire/src/pg_protocol.rs | 33 +++++-- src/utils/pgwire/src/pg_server.rs | 51 ++++++++-- 16 files changed, 284 insertions(+), 77 deletions(-) diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index 9c8db9dde51ee..3e6ebc7ef4322 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -282,7 +282,12 @@ impl TestCase { .chain(std::iter::once(self.sql())) { result = self - .run_sql(sql, session.clone(), do_check_result, result) + .run_sql( + Arc::from(sql.to_owned()), + session.clone(), + do_check_result, + result, + ) .await?; } @@ -326,7 +331,7 @@ impl TestCase { ); let temp_file = create_proto_file(content.as_str()); self.run_sql( - &(sql + temp_file.path().to_str().unwrap() + "')"), + Arc::from(sql + temp_file.path().to_str().unwrap() + "')"), session.clone(), false, None, @@ -357,7 +362,7 @@ impl TestCase { ); let temp_file = create_proto_file(content.as_str()); self.run_sql( - &(sql + temp_file.path().to_str().unwrap() + "')"), + Arc::from(sql + temp_file.path().to_str().unwrap() + "')"), session.clone(), false, None, @@ -376,15 +381,15 @@ impl TestCase { async fn run_sql( &self, - sql: &str, + sql: Arc, session: Arc, do_check_result: bool, mut result: Option, ) -> Result> { - let statements = Parser::parse_sql(sql).unwrap(); + let statements = Parser::parse_sql(&sql).unwrap(); for stmt in statements { // TODO: `sql` may contain multiple statements here. - let handler_args = HandlerArgs::new(session.clone(), &stmt, sql)?; + let handler_args = HandlerArgs::new(session.clone(), &stmt, sql.clone())?; let _guard = session.txn_begin_implicit(); match stmt.clone() { Statement::Query(_) @@ -399,7 +404,7 @@ impl TestCase { ..Default::default() }; let context = OptimizerContext::new( - HandlerArgs::new(session.clone(), &stmt, sql)?, + HandlerArgs::new(session.clone(), &stmt, sql.clone())?, explain_options, ); let ret = self.apply_query(&stmt, context.into())?; diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index 1b3babc41ceaf..f75c4043290df 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + use anyhow::Context; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; @@ -172,7 +174,7 @@ pub async fn handle_alter_table_column( } // Create handler args as if we're creating a new table with the altered definition. - let handler_args = HandlerArgs::new(session.clone(), &definition, "")?; + let handler_args = HandlerArgs::new(session.clone(), &definition, Arc::from(""))?; let col_id_gen = ColumnIdGenerator::new_alter(&original_catalog); let Statement::CreateTable { columns, diff --git a/src/frontend/src/handler/extended_handle.rs b/src/frontend/src/handler/extended_handle.rs index 1c0f0d36f0cbf..d6f22984f404e 100644 --- a/src/frontend/src/handler/extended_handle.rs +++ b/src/frontend/src/handler/extended_handle.rs @@ -57,11 +57,7 @@ impl std::fmt::Display for Portal { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { match &self { Portal::Empty => write!(f, "Empty"), - Portal::Portal(portal) => write!( - f, - "{}, params = {:?}", - portal.statement, portal.bound_result.parsed_params - ), + Portal::Portal(portal) => portal.fmt(f), Portal::PureStatement(stmt) => write!(f, "{}", stmt), } } @@ -74,14 +70,24 @@ pub struct PortalResult { pub result_formats: Vec, } +impl std::fmt::Display for PortalResult { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!( + f, + "{}, params = {:?}", + self.statement, self.bound_result.parsed_params + ) + } +} + pub fn handle_parse( session: Arc, statement: Statement, specific_param_types: Vec>, ) -> Result { session.clear_cancel_query_flag(); - let str_sql = statement.to_string(); - let handler_args = HandlerArgs::new(session, &statement, &str_sql)?; + let sql: Arc = Arc::from(statement.to_string()); + let handler_args = HandlerArgs::new(session, &statement, sql)?; match &statement { Statement::Query(_) | Statement::Insert { .. } @@ -181,8 +187,8 @@ pub async fn handle_execute(session: Arc, portal: Portal) -> Result Portal::Portal(portal) => { session.clear_cancel_query_flag(); let _guard = session.txn_begin_implicit(); // TODO(bugen): is this behavior correct? - let str_sql = portal.statement.to_string(); - let handler_args = HandlerArgs::new(session, &portal.statement, &str_sql)?; + let sql: Arc = Arc::from(portal.statement.to_string()); + let handler_args = HandlerArgs::new(session, &portal.statement, sql)?; match &portal.statement { Statement::Query(_) | Statement::Insert { .. } @@ -192,8 +198,8 @@ pub async fn handle_execute(session: Arc, portal: Portal) -> Result } } Portal::PureStatement(stmt) => { - let sql = stmt.to_string(); - handle(session, stmt, &sql, vec![]).await + let sql: Arc = Arc::from(stmt.to_string()); + handle(session, stmt, sql, vec![]).await } } } diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 748269a0bd58a..de8f048660f4a 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -110,16 +110,16 @@ impl From> for PgResponseStream { #[derive(Clone)] pub struct HandlerArgs { pub session: Arc, - pub sql: String, + pub sql: Arc, pub normalized_sql: String, pub with_options: WithOptions, } impl HandlerArgs { - pub fn new(session: Arc, stmt: &Statement, sql: &str) -> Result { + pub fn new(session: Arc, stmt: &Statement, sql: Arc) -> Result { Ok(Self { session, - sql: sql.into(), + sql, with_options: WithOptions::try_from(stmt)?, normalized_sql: Self::normalize_sql(stmt), }) @@ -172,12 +172,11 @@ impl HandlerArgs { pub async fn handle( session: Arc, stmt: Statement, - sql: &str, + sql: Arc, formats: Vec, ) -> Result { session.clear_cancel_query_flag(); let _guard = session.txn_begin_implicit(); - let handler_args = HandlerArgs::new(session, &stmt, sql)?; match stmt { diff --git a/src/frontend/src/handler/show.rs b/src/frontend/src/handler/show.rs index 88a9b1e694e33..2957519e39dff 100644 --- a/src/frontend/src/handler/show.rs +++ b/src/frontend/src/handler/show.rs @@ -16,7 +16,9 @@ use std::sync::Arc; use itertools::Itertools; use pgwire::pg_field_descriptor::PgFieldDescriptor; +use pgwire::pg_protocol::truncated_fmt; use pgwire::pg_response::{PgResponse, StatementType}; +use pgwire::pg_server::Session; use pgwire::types::Row; use risingwave_common::catalog::{ColumnCatalog, DEFAULT_SCHEMA_NAME}; use risingwave_common::error::{ErrorCode, Result}; @@ -267,6 +269,32 @@ pub async fn handle_show_object( .values(rows.into(), row_desc) .into()); } + ShowObject::ProcessList => { + let rows = { + let sessions_map = session.env().sessions_map(); + sessions_map + .read() + .values() + .map(|s| { + Row::new(vec![ + Some(format!("{}-{}", s.id().0, s.id().1).into()), + Some(s.user_name().to_owned().into()), + Some(format!("{}", s.peer_addr()).into()), + Some(s.database().to_owned().into()), + s.elapse_since_running_sql() + .map(|mills| format!("{}ms", mills).into()), + s.running_sql().map(|sql| { + format!("{}", truncated_fmt::TruncatedFmt(&sql, 1024)).into() + }), + ]) + }) + .collect_vec() + }; + + return Ok(PgResponse::builder(StatementType::SHOW_COMMAND) + .values(rows.into(), row_desc) + .into()); + } }; let rows = names diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index dcb4b74464b37..e4b8d3c566813 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -33,7 +33,7 @@ pub struct OptimizerContext { /// Store plan node id next_plan_node_id: RefCell, /// The original SQL string, used for debugging. - sql: String, + sql: Arc, /// Normalized SQL string. See [`HandlerArgs::normalize_sql`]. normalized_sql: String, /// Explain options @@ -97,7 +97,7 @@ impl OptimizerContext { Self { session_ctx: Arc::new(SessionImpl::mock()), next_plan_node_id: RefCell::new(0), - sql: "".to_owned(), + sql: Arc::from(""), normalized_sql: "".to_owned(), explain_options: ExplainOptions::default(), optimizer_trace: RefCell::new(vec![]), diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 0170057024621..201882cc6416c 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -14,16 +14,22 @@ use std::collections::HashMap; use std::io::{Error, ErrorKind}; +#[cfg(test)] +use std::net::{IpAddr, Ipv4Addr, SocketAddr}; use std::sync::atomic::{AtomicI32, Ordering}; -use std::sync::Arc; -use std::time::Duration; +use std::sync::{Arc, Weak}; +use std::time::{Duration, Instant}; use bytes::Bytes; use parking_lot::{Mutex, RwLock, RwLockReadGuard}; +use pgwire::net::{Address, AddressRef}; use pgwire::pg_field_descriptor::PgFieldDescriptor; use pgwire::pg_message::TransactionStatus; use pgwire::pg_response::PgResponse; -use pgwire::pg_server::{BoxedError, Session, SessionId, SessionManager, UserAuthenticator}; +use pgwire::pg_server::{ + BoxedError, ExecContext, ExecContextGuard, Session, SessionId, SessionManager, + UserAuthenticator, +}; use pgwire::types::{Format, FormatIterator}; use rand::RngCore; use risingwave_batch::task::{ShutdownSender, ShutdownToken}; @@ -132,7 +138,7 @@ pub struct FrontendEnv { } /// Session map identified by `(process_id, secret_key)` -type SessionMapRef = Arc>>>; +type SessionMapRef = Arc>>>; impl FrontendEnv { pub fn mock() -> Self { @@ -169,7 +175,7 @@ impl FrontendEnv { hummock_snapshot_manager, server_addr, client_pool, - sessions_map: Arc::new(Mutex::new(HashMap::new())), + sessions_map: Arc::new(RwLock::new(HashMap::new())), frontend_metrics: Arc::new(FrontendMetrics::for_test()), batch_config: BatchConfig::default(), meta_config: MetaConfig::default(), @@ -329,7 +335,7 @@ impl FrontendEnv { server_addr: frontend_address, client_pool, frontend_metrics, - sessions_map: Arc::new(Mutex::new(HashMap::new())), + sessions_map: Arc::new(RwLock::new(HashMap::new())), batch_config, meta_config, source_metrics, @@ -415,6 +421,10 @@ impl FrontendEnv { &self.creating_streaming_job_tracker } + pub fn sessions_map(&self) -> &SessionMapRef { + &self.sessions_map + } + pub fn compute_runtime(&self) -> Arc { self.compute_runtime.clone() } @@ -450,7 +460,7 @@ impl AuthContext { pub struct SessionImpl { env: FrontendEnv, auth_context: Arc, - // Used for user authentication. + /// Used for user authentication. user_authenticator: UserAuthenticator, /// Stores the value of configurations. config_map: Arc>, @@ -460,15 +470,21 @@ pub struct SessionImpl { /// Identified by process_id, secret_key. Corresponds to SessionManager. id: (i32, i32), + /// Client address + peer_addr: AddressRef, + /// Transaction state. - // TODO: get rid of the `Mutex` here as a workaround if the `Send` requirement of - // async functions, there should actually be no contention. + /// TODO: get rid of the `Mutex` here as a workaround if the `Send` requirement of + /// async functions, there should actually be no contention. txn: Arc>, /// Query cancel flag. /// This flag is set only when current query is executed in local mode, and used to cancel /// local query. current_query_cancel_flag: Mutex>, + + /// execution context represents the lifetime of a running SQL in the current session + exec_context: Mutex>>, } #[derive(Error, Debug)] @@ -494,6 +510,7 @@ impl SessionImpl { auth_context: Arc, user_authenticator: UserAuthenticator, id: SessionId, + peer_addr: AddressRef, ) -> Self { Self { env, @@ -501,9 +518,11 @@ impl SessionImpl { user_authenticator, config_map: Default::default(), id, + peer_addr, txn: Default::default(), current_query_cancel_flag: Mutex::new(None), notices: Default::default(), + exec_context: Mutex::new(None), } } @@ -523,6 +542,12 @@ impl SessionImpl { txn: Default::default(), current_query_cancel_flag: Mutex::new(None), notices: Default::default(), + exec_context: Mutex::new(None), + peer_addr: Address::Tcp(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), + 8080, + )) + .into(), } } @@ -571,6 +596,26 @@ impl SessionImpl { self.id } + pub fn running_sql(&self) -> Option> { + self.exec_context + .lock() + .as_ref() + .and_then(|weak| weak.upgrade()) + .map(|context| context.running_sql.clone()) + } + + pub fn peer_addr(&self) -> &Address { + &self.peer_addr + } + + pub fn elapse_since_running_sql(&self) -> Option { + self.exec_context + .lock() + .as_ref() + .and_then(|weak| weak.upgrade()) + .map(|context| context.last_instant.elapsed().as_millis()) + } + pub fn check_relation_name_duplicated( &self, name: ObjectName, @@ -710,11 +755,11 @@ impl SessionImpl { /// Maybe we can remove it in the future. pub async fn run_statement( self: Arc, - sql: &str, + sql: Arc, formats: Vec, ) -> std::result::Result, BoxedError> { // Parse sql. - let mut stmts = Parser::parse_sql(sql) + let mut stmts = Parser::parse_sql(&sql) .inspect_err(|e| tracing::error!("failed to parse sql:\n{}:\n{}", sql, e))?; if stmts.is_empty() { return Ok(PgResponse::empty_result( @@ -730,7 +775,7 @@ impl SessionImpl { } let stmt = stmts.swap_remove(0); let rsp = { - let mut handle_fut = Box::pin(handle(self, stmt, sql, formats)); + let mut handle_fut = Box::pin(handle(self, stmt, sql.clone(), formats)); if cfg!(debug_assertions) { // Report the SQL in the log periodically if the query is slow. const SLOW_QUERY_LOG_PERIOD: Duration = Duration::from_secs(60); @@ -740,7 +785,7 @@ impl SessionImpl { Ok(result) => break result, Err(_) => tracing::warn!( target: SLOW_QUERY_LOG, - sql, + sql = sql.as_ref(), "slow query has been running for another {SLOW_QUERY_LOG_PERIOD:?}" ), } @@ -782,6 +827,7 @@ impl SessionManager for SessionManagerImpl { &self, database: &str, user_name: &str, + peer_addr: AddressRef, ) -> std::result::Result, BoxedError> { let catalog_reader = self.env.catalog_reader(); let reader = catalog_reader.read_guard(); @@ -849,6 +895,7 @@ impl SessionManager for SessionManagerImpl { )), user_authenticator, id, + peer_addr, ) .into(); self.insert_session(session_impl.clone()); @@ -864,7 +911,7 @@ impl SessionManager for SessionManagerImpl { /// Used when cancel request happened. fn cancel_queries_in_session(&self, session_id: SessionId) { - let guard = self.env.sessions_map.lock(); + let guard = self.env.sessions_map.read(); if let Some(session) = guard.get(&session_id) { session.cancel_current_query() } else { @@ -873,7 +920,7 @@ impl SessionManager for SessionManagerImpl { } fn cancel_creating_jobs_in_session(&self, session_id: SessionId) { - let guard = self.env.sessions_map.lock(); + let guard = self.env.sessions_map.read(); if let Some(session) = guard.get(&session_id) { session.cancel_current_creating_job() } else { @@ -899,7 +946,7 @@ impl SessionManagerImpl { fn insert_session(&self, session: Arc) { let active_sessions = { - let mut write_guard = self.env.sessions_map.lock(); + let mut write_guard = self.env.sessions_map.write(); write_guard.insert(session.id(), session); write_guard.len() }; @@ -911,7 +958,7 @@ impl SessionManagerImpl { fn delete_session(&self, session_id: &SessionId) { let active_sessions = { - let mut write_guard = self.env.sessions_map.lock(); + let mut write_guard = self.env.sessions_map.write(); write_guard.remove(session_id); write_guard.len() }; @@ -934,9 +981,11 @@ impl Session for SessionImpl { stmt: Statement, format: Format, ) -> std::result::Result, BoxedError> { - let sql_str = stmt.to_string(); + let string = stmt.to_string(); + let sql_str = string.as_str(); + let sql: Arc = Arc::from(sql_str); let rsp = { - let mut handle_fut = Box::pin(handle(self, stmt, &sql_str, vec![format])); + let mut handle_fut = Box::pin(handle(self, stmt, sql.clone(), vec![format])); if cfg!(debug_assertions) { // Report the SQL in the log periodically if the query is slow. const SLOW_QUERY_LOG_PERIOD: Duration = Duration::from_secs(60); @@ -953,7 +1002,7 @@ impl Session for SessionImpl { handle_fut.await } } - .inspect_err(|e| tracing::error!("failed to handle sql:\n{}:\n{}", sql_str, e))?; + .inspect_err(|e| tracing::error!("failed to handle sql:\n{}:\n{}", sql, e))?; Ok(rsp) } @@ -1072,6 +1121,16 @@ impl Session for SessionImpl { // TODO: failed transaction } } + + /// Init and return an `ExecContextGuard` which could be used as a guard to represent the execution flow. + fn init_exec_context(&self, sql: Arc) -> ExecContextGuard { + let exec_context = Arc::new(ExecContext { + running_sql: sql, + last_instant: Instant::now(), + }); + *self.exec_context.lock() = Some(Arc::downgrade(&exec_context)); + ExecContextGuard::new(exec_context) + } } /// Returns row description of the statement diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index c2c6840c8dd77..4e336b29c002d 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -14,11 +14,13 @@ use std::collections::HashMap; use std::io::Write; +use std::net::{IpAddr, Ipv4Addr, SocketAddr}; use std::sync::atomic::{AtomicU32, Ordering}; use std::sync::Arc; use futures_async_stream::for_await; use parking_lot::RwLock; +use pgwire::net::{Address, AddressRef}; use pgwire::pg_response::StatementType; use pgwire::pg_server::{BoxedError, SessionId, SessionManager, UserAuthenticator}; use pgwire::types::Row; @@ -76,6 +78,7 @@ impl SessionManager for LocalFrontend { &self, _database: &str, _user_name: &str, + _peer_addr: AddressRef, ) -> std::result::Result, BoxedError> { Ok(self.session_ref()) } @@ -104,8 +107,8 @@ impl LocalFrontend { &self, sql: impl Into, ) -> std::result::Result> { - let sql = sql.into(); - self.session_ref().run_statement(sql.as_str(), vec![]).await + let sql: Arc = Arc::from(sql.into()); + self.session_ref().run_statement(sql, vec![]).await } pub async fn run_sql_with_session( @@ -113,8 +116,8 @@ impl LocalFrontend { session_ref: Arc, sql: impl Into, ) -> std::result::Result> { - let sql = sql.into(); - session_ref.run_statement(sql.as_str(), vec![]).await + let sql: Arc = Arc::from(sql.into()); + session_ref.run_statement(sql, vec![]).await } pub async fn run_user_sql( @@ -124,9 +127,9 @@ impl LocalFrontend { user_name: String, user_id: UserId, ) -> std::result::Result> { - let sql = sql.into(); + let sql: Arc = Arc::from(sql.into()); self.session_user_ref(database, user_name, user_id) - .run_statement(sql.as_str(), vec![]) + .run_statement(sql, vec![]) .await } @@ -178,6 +181,11 @@ impl LocalFrontend { UserAuthenticator::None, // Local Frontend use a non-sense id. (0, 0), + Address::Tcp(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), + 6666, + )) + .into(), )) } } diff --git a/src/frontend/src/utils/infer_stmt_row_desc.rs b/src/frontend/src/utils/infer_stmt_row_desc.rs index 8ebb7ac5c7d7a..2dc4115e305c0 100644 --- a/src/frontend/src/utils/infer_stmt_row_desc.rs +++ b/src/frontend/src/utils/infer_stmt_row_desc.rs @@ -161,6 +161,38 @@ pub fn infer_show_object(objects: &ShowObject) -> Vec { DataType::Varchar.type_len(), ), ], + ShowObject::ProcessList => vec![ + PgFieldDescriptor::new( + "Id".to_owned(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + PgFieldDescriptor::new( + "User".to_owned(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + PgFieldDescriptor::new( + "Host".to_owned(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + PgFieldDescriptor::new( + "Database".to_owned(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + PgFieldDescriptor::new( + "Time".to_owned(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + PgFieldDescriptor::new( + "Info".to_owned(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + ], _ => vec![PgFieldDescriptor::new( "Name".to_owned(), DataType::Varchar.to_oid(), diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index d96016334b72c..757fb7f2a237b 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -858,6 +858,7 @@ pub enum ShowObject { Indexes { table: ObjectName }, Cluster, Jobs, + ProcessList, } #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -899,6 +900,7 @@ impl fmt::Display for ShowObject { write!(f, "CLUSTER") } ShowObject::Jobs => write!(f, "JOBS"), + ShowObject::ProcessList => write!(f, "PROCESSLIST"), } } } diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index 4188f06f76ae3..4efdce19605cd 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -383,6 +383,7 @@ define_keywords!( PRIMARY, PRIVILEGES, PROCEDURE, + PROCESSLIST, PURGE, RANGE, RANK, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 70050d920cdf3..85932fb65e5f3 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -4090,6 +4090,12 @@ impl Parser { filter: self.parse_show_statement_filter()?, }); } + Keyword::PROCESSLIST => { + return Ok(Statement::ShowObjects { + object: ShowObject::ProcessList, + filter: self.parse_show_statement_filter()?, + }); + } _ => {} } } diff --git a/src/tests/sqlsmith/tests/frontend/mod.rs b/src/tests/sqlsmith/tests/frontend/mod.rs index 8f681ab38a956..ec34c83a4405d 100644 --- a/src/tests/sqlsmith/tests/frontend/mod.rs +++ b/src/tests/sqlsmith/tests/frontend/mod.rs @@ -44,7 +44,7 @@ pub struct SqlsmithEnv { /// Returns `Ok(true)` if query result was ignored. /// Skip status is required, so that we know if a SQL statement writing to the database was skipped. /// Then, we can infer the correct state of the database. -async fn handle(session: Arc, stmt: Statement, sql: &str) -> Result { +async fn handle(session: Arc, stmt: Statement, sql: Arc) -> Result { let result = handler::handle(session.clone(), stmt, sql, vec![]) .await .map(|_| ()) @@ -97,18 +97,19 @@ async fn create_tables( let (mut tables, statements) = parse_create_table_statements(sql); for s in statements { - let create_sql = s.to_string(); - handle(session.clone(), s, &create_sql).await?; + let create_sql: Arc = Arc::from(s.to_string()); + handle(session.clone(), s, create_sql).await?; } // Generate some mviews for i in 0..20 { let (sql, table) = mview_sql_gen(rng, tables.clone(), &format!("m{}", i)); + let sql: Arc = Arc::from(sql); reproduce_failing_queries(&setup_sql, &sql); setup_sql.push_str(&format!("{};", &sql)); let stmts = parse_sql(&sql); let stmt = stmts[0].clone(); - let skipped = handle(session.clone(), stmt, &sql).await?; + let skipped = handle(session.clone(), stmt, sql).await?; if !skipped { tables.push(table); } @@ -158,15 +159,16 @@ async fn test_stream_query( } let (sql, table) = mview_sql_gen(&mut rng, tables.clone(), "stream_query"); + let sql: Arc = Arc::from(sql); reproduce_failing_queries(setup_sql, &sql); // The generated SQL must be parsable. let stmt = round_trip_parse_test(&sql)?; - let skipped = handle(session.clone(), stmt, &sql).await?; + let skipped = handle(session.clone(), stmt, sql).await?; if !skipped { - let drop_sql = format!("DROP MATERIALIZED VIEW {}", table.name); + let drop_sql: Arc = Arc::from(format!("DROP MATERIALIZED VIEW {}", table.name)); let drop_stmts = parse_sql(&drop_sql); let drop_stmt = drop_stmts[0].clone(); - handle(session.clone(), drop_stmt, &drop_sql).await?; + handle(session.clone(), drop_stmt, drop_sql).await?; } Ok(()) } @@ -215,13 +217,13 @@ fn test_batch_query( rng = SmallRng::seed_from_u64(seed); } - let sql = sql_gen(&mut rng, tables); + let sql: Arc = Arc::from(sql_gen(&mut rng, tables)); reproduce_failing_queries(setup_sql, &sql); // The generated SQL must be parsable. let stmt = round_trip_parse_test(&sql)?; let context: OptimizerContextRef = - OptimizerContext::from_handler_args(HandlerArgs::new(session.clone(), &stmt, &sql)?).into(); + OptimizerContext::from_handler_args(HandlerArgs::new(session.clone(), &stmt, sql)?).into(); match stmt { Statement::Query(_) => { diff --git a/src/utils/pgwire/src/net.rs b/src/utils/pgwire/src/net.rs index ce341dec3e742..7b2d9f76d8ebf 100644 --- a/src/utils/pgwire/src/net.rs +++ b/src/utils/pgwire/src/net.rs @@ -16,6 +16,7 @@ use std::io; use std::net::SocketAddr as IpSocketAddr; #[cfg(madsim)] use std::os::unix::net::SocketAddr as UnixSocketAddr; +use std::sync::Arc; #[cfg(not(madsim))] use tokio::net::unix::SocketAddr as UnixSocketAddr; @@ -35,11 +36,13 @@ pub(crate) enum Stream { } /// A wrapper of either [`std::net::SocketAddr`] or [`tokio::net::unix::SocketAddr`]. -pub(crate) enum Address { +pub enum Address { Tcp(IpSocketAddr), Unix(UnixSocketAddr), } +pub type AddressRef = Arc
; + impl std::fmt::Display for Address { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { diff --git a/src/utils/pgwire/src/pg_protocol.rs b/src/utils/pgwire/src/pg_protocol.rs index c1ff5db59be64..8bdbd90d17360 100644 --- a/src/utils/pgwire/src/pg_protocol.rs +++ b/src/utils/pgwire/src/pg_protocol.rs @@ -35,6 +35,7 @@ use tokio_openssl::SslStream; use tracing::{error, warn, Instrument}; use crate::error::{PsqlError, PsqlResult}; +use crate::net::AddressRef; use crate::pg_extended::ResultCache; use crate::pg_message::{ BeCommandCompleteMessage, BeMessage, BeParameterStatusMessage, FeBindMessage, FeCancelMessage, @@ -90,6 +91,9 @@ where // Used in extended query protocol. When encounter error in extended query, we need to ignore // the following message util sync message. ignore_util_sync: bool, + + // Client Address + peer_addr: AddressRef, } const PGWIRE_QUERY_LOG: &str = "pgwire_query_log"; @@ -154,7 +158,12 @@ where S: AsyncWrite + AsyncRead + Unpin, SM: SessionManager, { - pub fn new(stream: S, session_mgr: Arc, tls_config: Option) -> Self { + pub fn new( + stream: S, + session_mgr: Arc, + tls_config: Option, + peer_addr: AddressRef, + ) -> Self { Self { stream: Conn::Unencrypted(PgStream { stream: Some(stream), @@ -174,6 +183,7 @@ where portal_store: Default::default(), statement_portal_dependency: Default::default(), ignore_util_sync: false, + peer_addr, } } @@ -366,7 +376,7 @@ where let session = self .session_mgr - .connect(&db_name, &user_name) + .connect(&db_name, &user_name, self.peer_addr.clone()) .map_err(PsqlError::StartupError)?; let application_name = msg.config.get("application_name"); @@ -429,11 +439,15 @@ where } async fn process_query_msg(&mut self, query_string: io::Result<&str>) -> PsqlResult<()> { - let sql = query_string.map_err(|err| PsqlError::QueryError(Box::new(err)))?; + let sql: Arc = + Arc::from(query_string.map_err(|err| PsqlError::QueryError(Box::new(err)))?); let start = Instant::now(); let session = self.session.clone().unwrap(); let session_id = session.id().0; - let result = self.inner_process_query_msg(sql, session).await; + let _exec_context_guard = session.init_exec_context(sql.clone()); + let result = self + .inner_process_query_msg(sql.clone(), session.clone()) + .await; let mills = start.elapsed().as_millis(); @@ -451,11 +465,11 @@ where async fn inner_process_query_msg( &mut self, - sql: &str, + sql: Arc, session: Arc, ) -> PsqlResult<()> { // Parse sql. - let stmts = Parser::parse_sql(sql) + let stmts = Parser::parse_sql(&sql) .inspect_err(|e| tracing::error!("failed to parse sql:\n{}:\n{}", sql, e)) .map_err(|err| PsqlError::QueryError(err.into()))?; if stmts.is_empty() { @@ -700,9 +714,10 @@ where } else { let start = Instant::now(); let portal = self.get_portal(&portal_name)?; - let sql = format!("{}", portal); + let sql: Arc = Arc::from(format!("{}", portal)); - let result = session.execute(portal).await; + let _exec_context_guard = session.init_exec_context(sql.clone()); + let result = session.clone().execute(portal).await; let mills = start.elapsed().as_millis(); @@ -1043,7 +1058,7 @@ fn build_ssl_ctx_from_config(tls_config: &TlsConfig) -> PsqlResult { Ok(acceptor.into_context()) } -mod truncated_fmt { +pub mod truncated_fmt { use std::fmt::*; struct TruncatedFormatter<'a, 'b> { diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index 2734ff857735e..f561540797b1a 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -16,13 +16,14 @@ use std::future::Future; use std::io; use std::result::Result; use std::sync::Arc; +use std::time::Instant; use bytes::Bytes; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::Statement; use tokio::io::{AsyncRead, AsyncWrite}; -use crate::net::Listener; +use crate::net::{AddressRef, Listener}; use crate::pg_field_descriptor::PgFieldDescriptor; use crate::pg_message::TransactionStatus; use crate::pg_protocol::{PgProtocol, TlsConfig}; @@ -37,7 +38,12 @@ pub type SessionId = (i32, i32); pub trait SessionManager: Send + Sync + 'static { type Session: Session; - fn connect(&self, database: &str, user_name: &str) -> Result, BoxedError>; + fn connect( + &self, + database: &str, + user_name: &str, + peer_addr: AddressRef, + ) -> Result, BoxedError>; fn cancel_queries_in_session(&self, session_id: SessionId); @@ -57,7 +63,7 @@ pub trait Session: Send + Sync { /// view, see . fn run_one_query( self: Arc, - sql: Statement, + stmt: Statement, format: Format, ) -> impl Future, BoxedError>> + Send; @@ -101,6 +107,26 @@ pub trait Session: Send + Sync { fn set_config(&self, key: &str, value: Vec) -> Result<(), BoxedError>; fn transaction_status(&self) -> TransactionStatus; + + fn init_exec_context(&self, sql: Arc) -> ExecContextGuard; +} + +/// Each session could run different SQLs multiple times. +/// `ExecContext` represents the lifetime of a running SQL in the current session. +pub struct ExecContext { + pub running_sql: Arc, + /// The instant of the running sql + pub last_instant: Instant, +} + +/// `ExecContextGuard` holds a `Arc` pointer. Once `ExecContextGuard` is dropped, +/// the inner `Arc` should not be referred anymore, so that its `Weak` reference (used in `SessionImpl`) will be the same lifecycle of the running sql execution context. +pub struct ExecContextGuard(Arc); + +impl ExecContextGuard { + pub fn new(exec_context: Arc) -> Self { + Self(exec_context) + } } #[derive(Debug, Clone)] @@ -146,6 +172,7 @@ pub async fn pg_serve( stream, session_mgr.clone(), tls_config.clone(), + Arc::new(peer_addr), )); } @@ -163,11 +190,12 @@ pub async fn handle_connection( stream: S, session_mgr: Arc, tls_config: Option, + peer_addr: AddressRef, ) where S: AsyncWrite + AsyncRead + Unpin, SM: SessionManager, { - let mut pg_proto = PgProtocol::new(stream, session_mgr, tls_config); + let mut pg_proto = PgProtocol::new(stream, session_mgr, tls_config, peer_addr); loop { let msg = match pg_proto.read_message().await { Ok(msg) => msg, @@ -191,6 +219,7 @@ pub async fn handle_connection( mod tests { use std::error::Error; use std::sync::Arc; + use std::time::Instant; use bytes::Bytes; use futures::stream::BoxStream; @@ -203,7 +232,8 @@ mod tests { use crate::pg_message::TransactionStatus; use crate::pg_response::{PgResponse, RowSetResult, StatementType}; use crate::pg_server::{ - pg_serve, BoxedError, Session, SessionId, SessionManager, UserAuthenticator, + pg_serve, BoxedError, ExecContext, ExecContextGuard, Session, SessionId, SessionManager, + UserAuthenticator, }; use crate::types; use crate::types::Row; @@ -218,6 +248,7 @@ mod tests { &self, _database: &str, _user_name: &str, + _peer_addr: crate::net::AddressRef, ) -> Result, Box> { Ok(Arc::new(MockSession {})) } @@ -240,7 +271,7 @@ mod tests { async fn run_one_query( self: Arc, - _sql: Statement, + _stmt: Statement, _format: types::Format, ) -> Result>, BoxedError> { Ok(PgResponse::builder(StatementType::SELECT) @@ -329,6 +360,14 @@ mod tests { fn transaction_status(&self) -> TransactionStatus { TransactionStatus::Idle } + + fn init_exec_context(&self, sql: Arc) -> ExecContextGuard { + let exec_context = Arc::new(ExecContext { + running_sql: sql, + last_instant: Instant::now(), + }); + ExecContextGuard::new(exec_context) + } } async fn do_test_query(bind_addr: impl Into, pg_config: impl Into) { From 8fae5b5309d7e8e7cee6c0362b755d3e54fc507a Mon Sep 17 00:00:00 2001 From: thexia <37214832+thexiay@users.noreply.github.com> Date: Wed, 8 Nov 2023 22:17:52 +0800 Subject: [PATCH 15/77] fix: array contains does not support nested types (#13290) Co-authored-by: thexia --- src/expr/impl/src/scalar/array_contain.rs | 14 +++++++--- .../tests/testdata/input/array.yaml | 8 ++++++ .../tests/testdata/output/array.yaml | 14 +++++++++- src/frontend/src/expr/type_inference/func.rs | 26 +++++++++++++++++-- 4 files changed, 55 insertions(+), 7 deletions(-) diff --git a/src/expr/impl/src/scalar/array_contain.rs b/src/expr/impl/src/scalar/array_contain.rs index 72ce363b3f5ce..5eb7f6a861d2d 100644 --- a/src/expr/impl/src/scalar/array_contain.rs +++ b/src/expr/impl/src/scalar/array_contain.rs @@ -35,19 +35,25 @@ use risingwave_expr::function; /// f /// /// query I -/// select array[1,2,3] @> NULL; +/// select array[[[1,2],[3,4]],[[5,6],[7,8]]] @> array[2,3]; +/// ---- +/// t +/// +/// query I +/// select array[1,2,3] @> null; /// ---- /// NULL /// /// query I -/// select NULL @> array[3,4]; +/// select null @> array[3,4]; /// ---- /// NULL /// ``` #[function("array_contains(anyarray, anyarray) -> boolean")] fn array_contains(left: ListRef<'_>, right: ListRef<'_>) -> bool { - let set: HashSet<_> = left.iter().collect(); - right.iter().all(|item| set.contains(&item)) + let flatten = left.flatten(); + let set: HashSet<_> = flatten.iter().collect(); + right.flatten().iter().all(|item| set.contains(&item)) } #[function("array_contained(anyarray, anyarray) -> boolean")] diff --git a/src/frontend/planner_test/tests/testdata/input/array.yaml b/src/frontend/planner_test/tests/testdata/input/array.yaml index 0dbfb89c644fb..bab19465d46e0 100644 --- a/src/frontend/planner_test/tests/testdata/input/array.yaml +++ b/src/frontend/planner_test/tests/testdata/input/array.yaml @@ -149,6 +149,10 @@ sql: select array[1,2] @> array[2,3]; expected_outputs: - logical_plan +- name: array_contains(int[][], int[]) -> bool + sql: select array[[1,2]] @> array[2,3]; + expected_outputs: + - logical_plan - name: array_contains(int[], int) -> bool sql: select array[1,2] @> 2; expected_outputs: @@ -157,6 +161,10 @@ sql: select array[2,3] @> array['1']; expected_outputs: - binder_error +- name: array_contains(int[][], varchar[][]) -> bool + sql: select array[array[1,2]] @> array[array['2','3']]; + expected_outputs: + - binder_error - name: any contains(null, null) -> bool sql: select '{}' @> '{}'; expected_outputs: diff --git a/src/frontend/planner_test/tests/testdata/output/array.yaml b/src/frontend/planner_test/tests/testdata/output/array.yaml index 5c5b73ebc74d4..31e1bfcb88124 100644 --- a/src/frontend/planner_test/tests/testdata/output/array.yaml +++ b/src/frontend/planner_test/tests/testdata/output/array.yaml @@ -187,6 +187,11 @@ logical_plan: |- LogicalProject { exprs: [(Array(1:Int32, 2:Int32) @> Array(2:Int32, 3:Int32)) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } +- name: array_contains(int[][], int[]) -> bool + sql: select array[[1,2]] @> array[2,3]; + logical_plan: |- + LogicalProject { exprs: [(Array(Array(1:Int32, 2:Int32)) @> Array(2:Int32, 3:Int32)) as $expr1] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: array_contains(int[], int) -> bool sql: select array[1,2] @> 2; binder_error: |- @@ -200,7 +205,14 @@ Bind error: failed to bind expression: ARRAY[2, 3] @> ARRAY['1'] Caused by: - Bind error: types List(Int32) and List(Varchar) cannot be matched + Bind error: Cannot array_contains unnested type integer to unnested type character varying +- name: array_contains(int[][], varchar[][]) -> bool + sql: select array[array[1,2]] @> array[array['2','3']]; + binder_error: |- + Bind error: failed to bind expression: ARRAY[ARRAY[1, 2]] @> ARRAY[ARRAY['2', '3']] + + Caused by: + Bind error: Cannot array_contains unnested type integer to unnested type character varying - name: any contains(null, null) -> bool sql: select '{}' @> '{}'; binder_error: |- diff --git a/src/frontend/src/expr/type_inference/func.rs b/src/frontend/src/expr/type_inference/func.rs index 17a94812731eb..d760ee039499a 100644 --- a/src/frontend/src/expr/type_inference/func.rs +++ b/src/frontend/src/expr/type_inference/func.rs @@ -535,8 +535,30 @@ fn infer_type_for_special( } ExprType::ArrayContains | ExprType::ArrayContained => { ensure_arity!("array_contains/array_contained", | inputs | == 2); - align_types(inputs.iter_mut())?; - Ok(Some(DataType::Boolean)) + let left_type = (!inputs[0].is_untyped()).then(|| inputs[0].return_type()); + let right_type = (!inputs[1].is_untyped()).then(|| inputs[1].return_type()); + match (left_type, right_type) { + (None, Some(DataType::List(_))) | (Some(DataType::List(_)), None) => { + align_types(inputs.iter_mut())?; + Ok(Some(DataType::Boolean)) + } + (Some(DataType::List(left)), Some(DataType::List(right))) => { + // cannot directly cast, find unnest type and judge if they are same type + let left = left.unnest_list(); + let right = right.unnest_list(); + if left.equals_datatype(right) { + Ok(Some(DataType::Boolean)) + } else { + Err(ErrorCode::BindError(format!( + "Cannot array_contains unnested type {} to unnested type {}", + left, right + )) + .into()) + } + } + // any other condition cannot determine polymorphic type + _ => Ok(None), + } } ExprType::Vnode => { ensure_arity!("vnode", 1 <= | inputs |); From 61c3e2ca1954ced2e6278c0f1de3b8634f039023 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Wed, 8 Nov 2023 22:38:08 +0800 Subject: [PATCH 16/77] fix(udf): add graceful shutdown for python UDF server (#13285) Signed-off-by: Runji Wang --- ci/scripts/run-e2e-test.sh | 6 ++-- e2e_test/udf/graceful_shutdown_python.slt | 36 +++++++++++++++++++ e2e_test/udf/test.py | 10 +++++- .../src/main/java/com/example/UdfExample.java | 12 +++++++ src/udf/python/CHANGELOG.md | 14 ++++++++ src/udf/python/pyproject.toml | 2 +- src/udf/python/risingwave/udf.py | 8 ++++- 7 files changed, 83 insertions(+), 5 deletions(-) create mode 100644 e2e_test/udf/graceful_shutdown_python.slt create mode 100644 src/udf/python/CHANGELOG.md diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index 8fb29ec5bd1a2..d6d26e0236ad0 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -82,13 +82,15 @@ sqllogictest -p 4566 -d dev './e2e_test/superset/*.slt' --junit "batch-${profile echo "--- e2e, $mode, python udf" python3 e2e_test/udf/test.py & -sleep 2 +sleep 1 sqllogictest -p 4566 -d dev './e2e_test/udf/udf.slt' pkill python3 +sqllogictest -p 4566 -d dev './e2e_test/udf/graceful_shutdown_python.slt' + echo "--- e2e, $mode, java udf" java -jar risingwave-udf-example.jar & -sleep 2 +sleep 1 sqllogictest -p 4566 -d dev './e2e_test/udf/udf.slt' pkill java diff --git a/e2e_test/udf/graceful_shutdown_python.slt b/e2e_test/udf/graceful_shutdown_python.slt new file mode 100644 index 0000000000000..139a3fd284981 --- /dev/null +++ b/e2e_test/udf/graceful_shutdown_python.slt @@ -0,0 +1,36 @@ +system ok +python3 e2e_test/udf/test.py & + +# wait for server to start +sleep 1s + +statement ok +CREATE FUNCTION sleep(INT) RETURNS INT AS 'sleep' USING LINK 'http://localhost:8815'; + +system ok +sleep 1 && pkill python & + +# python server should not exit until the query is finished +query I +select sleep(2); +---- +0 + +# wait for server to exit +sleep 1s + +system ok +python3 e2e_test/udf/test.py & + +# wait for server to start +sleep 1s + +# force kill python server +system ok +sleep 1 && pkill -9 python & + +query error +select sleep(2); + +statement ok +DROP FUNCTION sleep; diff --git a/e2e_test/udf/test.py b/e2e_test/udf/test.py index c34c65e3c232c..45db54a8113b3 100644 --- a/e2e_test/udf/test.py +++ b/e2e_test/udf/test.py @@ -15,6 +15,7 @@ import socket import struct import sys +import time from typing import Iterator, List, Optional, Tuple, Any from decimal import Decimal @@ -28,6 +29,12 @@ def int_42() -> int: return 42 +@udf(input_types=["INT"], result_type="INT") +def sleep(s: int) -> int: + time.sleep(s) + return 0 + + @udf(input_types=["INT", "INT"], result_type="INT") def gcd(x: int, y: int) -> int: while y != 0: @@ -190,8 +197,9 @@ def return_all_arrays( if __name__ == "__main__": - server = UdfServer(location="0.0.0.0:8815") + server = UdfServer(location="localhost:8815") server.add_function(int_42) + server.add_function(sleep) server.add_function(gcd) server.add_function(gcd3) server.add_function(series) diff --git a/java/udf-example/src/main/java/com/example/UdfExample.java b/java/udf-example/src/main/java/com/example/UdfExample.java index eed88d3dda281..3f07f457fd81b 100644 --- a/java/udf-example/src/main/java/com/example/UdfExample.java +++ b/java/udf-example/src/main/java/com/example/UdfExample.java @@ -35,6 +35,7 @@ public class UdfExample { public static void main(String[] args) throws IOException { try (var server = new UdfServer("0.0.0.0", 8815)) { server.addFunction("int_42", new Int42()); + server.addFunction("sleep", new Sleep()); server.addFunction("gcd", new Gcd()); server.addFunction("gcd3", new Gcd3()); server.addFunction("extract_tcp_info", new ExtractTcpInfo()); @@ -62,6 +63,17 @@ public int eval() { } } + public static class Sleep implements ScalarFunction { + public int eval(int x) { + try { + Thread.sleep(x * 1000); + } catch (InterruptedException e) { + e.printStackTrace(); + } + return 0; + } + } + public static class Gcd implements ScalarFunction { public int eval(int a, int b) { while (b != 0) { diff --git a/src/udf/python/CHANGELOG.md b/src/udf/python/CHANGELOG.md new file mode 100644 index 0000000000000..3c788857a395b --- /dev/null +++ b/src/udf/python/CHANGELOG.md @@ -0,0 +1,14 @@ +# Changelog + +All notable changes to this project will be documented in this file. + +The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), +and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.html). + +## [Unreleased] + +## [0.0.11] - 2023-11-06 + +### Fixed + +- Hook SIGTERM to stop the UDF server gracefully. diff --git a/src/udf/python/pyproject.toml b/src/udf/python/pyproject.toml index d3093156eff56..de9b245175f9a 100644 --- a/src/udf/python/pyproject.toml +++ b/src/udf/python/pyproject.toml @@ -4,7 +4,7 @@ build-backend = "setuptools.build_meta" [project] name = "risingwave" -version = "0.0.10" +version = "0.0.11" authors = [{ name = "RisingWave Labs" }] description = "RisingWave Python API" readme = "README.md" diff --git a/src/udf/python/risingwave/udf.py b/src/udf/python/risingwave/udf.py index 758421b1db68d..ea7476a756d6c 100644 --- a/src/udf/python/risingwave/udf.py +++ b/src/udf/python/risingwave/udf.py @@ -21,6 +21,7 @@ import json from concurrent.futures import ThreadPoolExecutor import concurrent +import signal class UserDefinedFunction: @@ -392,11 +393,16 @@ def do_exchange(self, context, descriptor, reader, writer): raise e def serve(self): - """Start the server.""" + """ + Block until the server shuts down. + + This method only returns if shutdown() is called or a signal (SIGINT, SIGTERM) received. + """ print( "Note: You can use arbitrary function names and struct field names in CREATE FUNCTION statements." f"\n\nlistening on {self._location}" ) + signal.signal(signal.SIGTERM, lambda s, f: self.shutdown()) super(UdfServer, self).serve() From 306801bbb4bd8b0b1bf85a044d1229da68215bf1 Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Wed, 8 Nov 2023 23:10:56 +0800 Subject: [PATCH 17/77] feat: Introduce scale-in in recovery. (#13270) Signed-off-by: Shanicky Chen --- src/common/src/config.rs | 4 + src/config/example.toml | 1 + src/meta/node/src/lib.rs | 1 + src/meta/service/src/scale_service.rs | 13 +- src/meta/src/barrier/command.rs | 65 +-- src/meta/src/barrier/mod.rs | 12 +- src/meta/src/barrier/recovery.rs | 105 +++- src/meta/src/manager/env.rs | 3 + src/meta/src/stream/scale.rs | 464 ++++++++++-------- src/meta/src/stream/stream_manager.rs | 70 ++- src/tests/simulation/src/cluster.rs | 38 ++ .../simulation/src/risingwave-auto-scale.toml | 17 + .../tests/integration_tests/recovery/mod.rs | 1 + .../recovery/scale_in_when_recovery.rs | 166 +++++++ 14 files changed, 697 insertions(+), 263 deletions(-) create mode 100644 src/tests/simulation/src/risingwave-auto-scale.toml create mode 100644 src/tests/simulation/tests/integration_tests/recovery/scale_in_when_recovery.rs diff --git a/src/common/src/config.rs b/src/common/src/config.rs index ae92218803503..bba8f50d5af68 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -208,6 +208,10 @@ pub struct MetaConfig { #[serde(default)] pub disable_recovery: bool, + /// Whether to enable scale-in when recovery. + #[serde(default)] + pub enable_scale_in_when_recovery: bool, + #[serde(default = "default::meta::meta_leader_lease_secs")] pub meta_leader_lease_secs: u64, diff --git a/src/config/example.toml b/src/config/example.toml index 93d6a0820102b..61d9e23544a05 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -22,6 +22,7 @@ hummock_version_checkpoint_interval_sec = 30 min_delta_log_num_for_hummock_version_checkpoint = 10 max_heartbeat_interval_secs = 300 disable_recovery = false +enable_scale_in_when_recovery = false meta_leader_lease_secs = 30 default_parallelism = "Full" enable_compaction_deterministic = false diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index 2737c35e68eb2..dfd3cdf77d4b1 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -249,6 +249,7 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { config.meta.meta_leader_lease_secs, MetaOpts { enable_recovery: !config.meta.disable_recovery, + enable_scale_in_when_recovery: config.meta.enable_scale_in_when_recovery, in_flight_barrier_nums, max_idle_ms, compaction_deterministic_test: config.meta.enable_compaction_deterministic, diff --git a/src/meta/service/src/scale_service.rs b/src/meta/service/src/scale_service.rs index 676180adc7581..c054b9b18bd6d 100644 --- a/src/meta/service/src/scale_service.rs +++ b/src/meta/service/src/scale_service.rs @@ -12,6 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + +use risingwave_meta::stream::{ScaleController, ScaleControllerRef}; use risingwave_pb::common::WorkerType; use risingwave_pb::meta::scale_service_server::ScaleService; use risingwave_pb::meta::{ @@ -35,6 +38,7 @@ pub struct ScaleServiceImpl { catalog_manager: CatalogManagerRef, stream_manager: GlobalStreamManagerRef, barrier_manager: BarrierManagerRef, + scale_controller: ScaleControllerRef, } impl ScaleServiceImpl { @@ -46,6 +50,12 @@ impl ScaleServiceImpl { stream_manager: GlobalStreamManagerRef, barrier_manager: BarrierManagerRef, ) -> Self { + let scale_controller = Arc::new(ScaleController::new( + fragment_manager.clone(), + cluster_manager.clone(), + source_manager.clone(), + stream_manager.env.clone(), + )); Self { fragment_manager, cluster_manager, @@ -53,6 +63,7 @@ impl ScaleServiceImpl { catalog_manager, stream_manager, barrier_manager, + scale_controller, } } } @@ -203,7 +214,7 @@ impl ScaleService for ScaleServiceImpl { .policy .ok_or_else(|| Status::invalid_argument("policy is required"))?; - let plan = self.stream_manager.get_reschedule_plan(policy).await?; + let plan = self.scale_controller.get_reschedule_plan(policy).await?; let next_revision = self.fragment_manager.get_revision().await; diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 5ea20daccb85b..bc2692f4bad2a 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -39,7 +39,9 @@ use crate::barrier::CommandChanges; use crate::hummock::HummockManagerRef; use crate::manager::{CatalogManagerRef, FragmentManagerRef, WorkerId}; use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments}; -use crate::stream::{build_actor_connector_splits, SourceManagerRef, SplitAssignment}; +use crate::stream::{ + build_actor_connector_splits, ScaleControllerRef, SourceManagerRef, SplitAssignment, +}; use crate::MetaResult; /// [`Reschedule`] is for the [`Command::RescheduleFragment`], which is used for rescheduling actors @@ -238,6 +240,8 @@ pub struct CommandContext { source_manager: SourceManagerRef, + scale_controller: ScaleControllerRef, + /// The tracing span of this command. /// /// Differs from [`TracedEpoch`], this span focuses on the lifetime of the corresponding @@ -260,6 +264,7 @@ impl CommandContext { command: Command, kind: BarrierKind, source_manager: SourceManagerRef, + scale_controller: ScaleControllerRef, span: tracing::Span, ) -> Self { Self { @@ -274,6 +279,7 @@ impl CommandContext { command, kind, source_manager, + scale_controller, span, } } @@ -760,60 +766,11 @@ impl CommandContext { } Command::RescheduleFragment { reschedules } => { - let mut node_dropped_actors = HashMap::new(); - for table_fragments in self - .fragment_manager - .get_fragment_read_guard() - .await - .table_fragments() - .values() - { - for fragment_id in table_fragments.fragments.keys() { - if let Some(reschedule) = reschedules.get(fragment_id) { - for actor_id in &reschedule.removed_actors { - let node_id = table_fragments - .actor_status - .get(actor_id) - .unwrap() - .parallel_unit - .as_ref() - .unwrap() - .worker_node_id; - node_dropped_actors - .entry(node_id as WorkerId) - .or_insert(vec![]) - .push(*actor_id as ActorId); - } - } - } - } - self.clean_up(node_dropped_actors).await?; - - // Update fragment info after rescheduling in meta store. - self.fragment_manager - .post_apply_reschedules(reschedules.clone()) + let node_dropped_actors = self + .scale_controller + .post_apply_reschedule(reschedules) .await?; - - let mut stream_source_actor_splits = HashMap::new(); - let mut stream_source_dropped_actors = HashSet::new(); - - for (fragment_id, reschedule) in reschedules { - if !reschedule.actor_splits.is_empty() { - stream_source_actor_splits - .insert(*fragment_id as FragmentId, reschedule.actor_splits.clone()); - stream_source_dropped_actors.extend(reschedule.removed_actors.clone()); - } - } - - if !stream_source_actor_splits.is_empty() { - self.source_manager - .apply_source_change( - None, - Some(stream_source_actor_splits), - Some(stream_source_dropped_actors), - ) - .await; - } + self.clean_up(node_dropped_actors).await?; } Command::ReplaceTable { diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 654c6359ca1bf..acd90dce8f522 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -60,7 +60,7 @@ use crate::manager::{ }; use crate::model::{ActorId, BarrierManagerState, TableFragments}; use crate::rpc::metrics::MetaMetrics; -use crate::stream::SourceManagerRef; +use crate::stream::{ScaleController, ScaleControllerRef, SourceManagerRef}; use crate::{MetaError, MetaResult}; mod command; @@ -176,6 +176,8 @@ pub struct GlobalBarrierManager { source_manager: SourceManagerRef, + scale_controller: ScaleControllerRef, + sink_manager: SinkCoordinatorManager, metrics: Arc, @@ -529,6 +531,12 @@ impl GlobalBarrierManager { let in_flight_barrier_nums = env.opts.in_flight_barrier_nums; let tracker = CreateMviewProgressTracker::new(); + let scale_controller = Arc::new(ScaleController::new( + fragment_manager.clone(), + cluster_manager.clone(), + source_manager.clone(), + env.clone(), + )); Self { enable_recovery, status: Mutex::new(BarrierManagerStatus::Starting), @@ -539,6 +547,7 @@ impl GlobalBarrierManager { fragment_manager, hummock_manager, source_manager, + scale_controller, sink_manager, metrics, env, @@ -733,6 +742,7 @@ impl GlobalBarrierManager { command, kind, self.source_manager.clone(), + self.scale_controller.clone(), span.clone(), )); diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 3e319f0e69a52..fc18ba4fbb612 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -21,6 +21,7 @@ use futures::future::try_join_all; use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_pb::common::ActorInfo; +use risingwave_pb::meta::get_reschedule_plan_request::{PbWorkerChanges, StableResizePolicy}; use risingwave_pb::meta::PausedReason; use risingwave_pb::stream_plan::barrier::{BarrierKind, Mutation}; use risingwave_pb::stream_plan::AddMutation; @@ -40,7 +41,7 @@ use crate::barrier::progress::CreateMviewProgressTracker; use crate::barrier::{CheckpointControl, Command, GlobalBarrierManager}; use crate::manager::WorkerId; use crate::model::{BarrierManagerState, MigrationPlan}; -use crate::stream::build_actor_connector_splits; +use crate::stream::{build_actor_connector_splits, RescheduleOptions}; use crate::MetaResult; impl GlobalBarrierManager { @@ -254,12 +255,21 @@ impl GlobalBarrierManager { // following steps will be no-op, while the compute nodes will still be reset. let mut info = self.resolve_actor_info_for_recovery().await; - // Migrate actors in expired CN to newly joined one. - let migrated = self.migrate_actors(&info).await.inspect_err(|err| { - warn!(err = ?err, "migrate actors failed"); - })?; - if migrated { - info = self.resolve_actor_info_for_recovery().await; + if self.env.opts.enable_scale_in_when_recovery { + let scaled = self.scale_actors(&info).await.inspect_err(|err| { + warn!(err = ?err, "scale actors failed"); + })?; + if scaled { + info = self.resolve_actor_info_for_recovery().await; + } + } else { + // Migrate actors in expired CN to newly joined one. + let migrated = self.migrate_actors(&info).await.inspect_err(|err| { + warn!(err = ?err, "migrate actors failed"); + })?; + if migrated { + info = self.resolve_actor_info_for_recovery().await; + } } // Reset all compute nodes, stop and drop existing actors. @@ -301,6 +311,7 @@ impl GlobalBarrierManager { command, BarrierKind::Initial, self.source_manager.clone(), + self.scale_controller.clone(), tracing::Span::current(), // recovery span )); @@ -386,6 +397,86 @@ impl GlobalBarrierManager { Ok(true) } + async fn scale_actors(&self, info: &BarrierActorInfo) -> MetaResult { + debug!("start scaling-in offline actors."); + + let expired_workers: HashSet = info + .actor_map + .iter() + .filter(|(&worker, actors)| !actors.is_empty() && !info.node_map.contains_key(&worker)) + .map(|(&worker, _)| worker) + .collect(); + + if expired_workers.is_empty() { + debug!("no expired workers, skipping."); + return Ok(false); + } + + let all_worker_parallel_units = self.fragment_manager.all_worker_parallel_units().await; + + let expired_worker_parallel_units: HashMap<_, _> = all_worker_parallel_units + .into_iter() + .filter(|(worker, _)| expired_workers.contains(worker)) + .collect(); + + let fragment_worker_changes = { + let guard = self.fragment_manager.get_fragment_read_guard().await; + let mut policy = HashMap::new(); + for table_fragments in guard.table_fragments().values() { + for fragment_id in table_fragments.fragment_ids() { + policy.insert( + fragment_id, + PbWorkerChanges { + exclude_worker_ids: expired_workers.iter().cloned().collect(), + ..Default::default() + }, + ); + } + } + policy + }; + + let plan = self + .scale_controller + .generate_stable_resize_plan( + StableResizePolicy { + fragment_worker_changes, + }, + Some(expired_worker_parallel_units), + ) + .await?; + + let (reschedule_fragment, applied_reschedules) = self + .scale_controller + .prepare_reschedule_command( + plan, + RescheduleOptions { + resolve_no_shuffle_upstream: true, + }, + ) + .await?; + + if let Err(e) = self + .scale_controller + .post_apply_reschedule(&reschedule_fragment) + .await + { + tracing::error!( + "failed to apply reschedule for offline scaling in recovery: {}", + e.to_string() + ); + + self.fragment_manager + .cancel_apply_reschedules(applied_reschedules) + .await; + + return Err(e); + } + + debug!("scaling-in actors succeed."); + Ok(true) + } + /// This function will generate a migration plan, which includes the mapping for all expired and /// in-used parallel unit to a new one. async fn generate_migration_plan( diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 28d8200c73ea5..c9c22ff438cf8 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -81,6 +81,8 @@ pub struct MetaOpts { /// Whether to enable the recovery of the cluster. If disabled, the meta service will exit on /// abnormal cases. pub enable_recovery: bool, + /// Whether to enable the scale-in feature when compute-node is removed. + pub enable_scale_in_when_recovery: bool, /// The maximum number of barriers in-flight in the compute nodes. pub in_flight_barrier_nums: usize, /// After specified seconds of idle (no mview or flush), the process will be exited. @@ -174,6 +176,7 @@ impl MetaOpts { pub fn test(enable_recovery: bool) -> Self { Self { enable_recovery, + enable_scale_in_when_recovery: false, in_flight_barrier_nums: 40, max_idle_ms: 0, compaction_deterministic_test: false, diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 582c6585f5ba8..99a4d6fc76e92 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -15,9 +15,9 @@ use std::cmp::{min, Ordering}; use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet, VecDeque}; use std::iter::repeat; +use std::sync::Arc; use anyhow::{anyhow, Context}; -use futures::future::BoxFuture; use itertools::Itertools; use num_integer::Integer; use num_traits::abs; @@ -37,11 +37,11 @@ use risingwave_pb::stream_service::{ }; use uuid::Uuid; -use crate::barrier::{Command, Reschedule}; -use crate::manager::{IdCategory, WorkerId}; +use crate::barrier::Reschedule; +use crate::manager::{ClusterManagerRef, FragmentManagerRef, IdCategory, MetaSrvEnv, WorkerId}; use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments}; use crate::storage::{MetaStore, MetaStoreError, MetaStoreRef, Transaction, DEFAULT_COLUMN_FAMILY}; -use crate::stream::GlobalStreamManager; +use crate::stream::SourceManagerRef; use crate::{MetaError, MetaResult}; #[derive(Copy, Clone, Debug, Eq, PartialEq, Ord, PartialOrd)] @@ -92,11 +92,6 @@ pub struct ParallelUnitReschedule { pub removed_parallel_units: BTreeSet, } -#[derive(Debug, Clone, Copy)] -pub struct RescheduleOptions { - pub resolve_no_shuffle_upstream: bool, -} - pub struct RescheduleContext { /// Index used to map `ParallelUnitId` to `WorkerId` parallel_unit_id_to_worker_id: BTreeMap, @@ -346,7 +341,39 @@ pub fn rebalance_actor_vnode( result } -impl GlobalStreamManager { +#[derive(Debug, Clone, Copy)] +pub struct RescheduleOptions { + /// Whether to resolve the upstream of NoShuffle when scaling. It will check whether all the reschedules in the no shuffle dependency tree are corresponding, and rewrite them to the root of the no shuffle dependency tree. + pub resolve_no_shuffle_upstream: bool, +} + +pub type ScaleControllerRef = Arc; + +pub struct ScaleController { + pub(super) fragment_manager: FragmentManagerRef, + + pub cluster_manager: ClusterManagerRef, + + pub source_manager: SourceManagerRef, + + pub env: MetaSrvEnv, +} + +impl ScaleController { + pub fn new( + fragment_manager: FragmentManagerRef, + cluster_manager: ClusterManagerRef, + source_manager: SourceManagerRef, + env: MetaSrvEnv, + ) -> Self { + Self { + fragment_manager, + cluster_manager, + source_manager, + env, + } + } + /// Build the context for rescheduling and do some validation for the request. async fn build_reschedule_context( &self, @@ -622,123 +649,68 @@ impl GlobalStreamManager { }) } - fn resolve_no_shuffle_upstream( - reschedule: &mut HashMap, - fragment_map: &HashMap, - no_shuffle_source_fragment_ids: &HashSet, - no_shuffle_target_fragment_ids: &HashSet, - ) -> MetaResult<()> - where - T: Clone + Eq, - { - let mut queue: VecDeque = reschedule.keys().cloned().collect(); - - // We trace the upstreams of each downstream under the hierarchy until we reach the top - // for every no_shuffle relation. - while let Some(fragment_id) = queue.pop_front() { - if !no_shuffle_target_fragment_ids.contains(&fragment_id) - && !no_shuffle_source_fragment_ids.contains(&fragment_id) - { - continue; - } - - // for upstream - for upstream_fragment_id in &fragment_map - .get(&fragment_id) - .unwrap() - .upstream_fragment_ids - { - if !no_shuffle_source_fragment_ids.contains(upstream_fragment_id) { - continue; - } - - let reschedule_plan = reschedule.get(&fragment_id).unwrap(); - - if let Some(upstream_reschedule_plan) = reschedule.get(upstream_fragment_id) { - if upstream_reschedule_plan != reschedule_plan { - bail!("Inconsistent NO_SHUFFLE plan, check target worker ids of fragment {} and {}", fragment_id, upstream_fragment_id); - } + async fn create_actors_on_compute_node( + &self, + worker_nodes: &HashMap, + actor_infos_to_broadcast: BTreeMap, + node_actors_to_create: HashMap>, + broadcast_worker_ids: HashSet, + ) -> MetaResult<()> { + for worker_id in &broadcast_worker_ids { + let node = worker_nodes.get(worker_id).unwrap(); + let client = self.env.stream_client_pool().get(node).await?; - continue; - } + let actor_infos_to_broadcast = actor_infos_to_broadcast.values().cloned().collect(); - reschedule.insert(*upstream_fragment_id, reschedule_plan.clone()); - queue.push_back(*upstream_fragment_id); - } + client + .to_owned() + .broadcast_actor_info_table(BroadcastActorInfoTableRequest { + info: actor_infos_to_broadcast, + }) + .await?; } - reschedule.retain(|fragment_id, _| !no_shuffle_target_fragment_ids.contains(fragment_id)); - - Ok(()) - } + for (node_id, stream_actors) in &node_actors_to_create { + let node = worker_nodes.get(node_id).unwrap(); + let client = self.env.stream_client_pool().get(node).await?; + let request_id = Uuid::new_v4().to_string(); + let request = UpdateActorsRequest { + request_id, + actors: stream_actors.clone(), + }; - fn build_fragment_dispatcher_index( - actor_map: &HashMap, - fragment_dispatcher_map: &mut HashMap>, - ) { - for actor in actor_map.values() { - for dispatcher in &actor.dispatcher { - for downstream_actor_id in &dispatcher.downstream_actor_id { - if let Some(downstream_actor) = actor_map.get(downstream_actor_id) { - fragment_dispatcher_map - .entry(actor.fragment_id as FragmentId) - .or_default() - .insert( - downstream_actor.fragment_id as FragmentId, - dispatcher.r#type(), - ); - } - } - } + client.to_owned().update_actors(request).await?; } - } - fn build_no_shuffle_relation_index( - actor_map: &HashMap, - no_shuffle_source_fragment_ids: &mut HashSet, - no_shuffle_target_fragment_ids: &mut HashSet, - ) { - for actor in actor_map.values() { - for dispatcher in &actor.dispatcher { - for downstream_actor_id in &dispatcher.downstream_actor_id { - if let Some(downstream_actor) = actor_map.get(downstream_actor_id) { - // Checking for no shuffle dispatchers - if dispatcher.r#type() == DispatcherType::NoShuffle { - no_shuffle_source_fragment_ids.insert(actor.fragment_id as FragmentId); - no_shuffle_target_fragment_ids - .insert(downstream_actor.fragment_id as FragmentId); - } - } - } - } - } - } + for (node_id, stream_actors) in node_actors_to_create { + let node = worker_nodes.get(&node_id).unwrap(); + let client = self.env.stream_client_pool().get(node).await?; + let request_id = Uuid::new_v4().to_string(); - pub async fn reschedule_actors( - &self, - reschedules: HashMap, - options: RescheduleOptions, - ) -> MetaResult<()> { - let mut revert_funcs = vec![]; - if let Err(e) = self - .reschedule_actors_impl(&mut revert_funcs, reschedules, options) - .await - { - for revert_func in revert_funcs.into_iter().rev() { - revert_func.await; - } - return Err(e); + client + .to_owned() + .build_actors(BuildActorsRequest { + request_id, + actor_id: stream_actors + .iter() + .map(|stream_actor| stream_actor.actor_id) + .collect(), + }) + .await?; } Ok(()) } - async fn reschedule_actors_impl( + // Results are the generated reschedule plan and the changes that need to be updated to the meta store. + pub(crate) async fn prepare_reschedule_command( &self, - revert_funcs: &mut Vec>, mut reschedules: HashMap, options: RescheduleOptions, - ) -> MetaResult<()> { + ) -> MetaResult<( + HashMap, + HashMap>, + )> { let ctx = self .build_reschedule_context(&mut reschedules, options) .await?; @@ -1297,78 +1269,7 @@ impl GlobalStreamManager { .pre_apply_reschedules(fragment_created_actors) .await; - let fragment_manager_ref = self.fragment_manager.clone(); - - revert_funcs.push(Box::pin(async move { - fragment_manager_ref - .cancel_apply_reschedules(applied_reschedules) - .await; - })); - - let _source_pause_guard = self.source_manager.paused.lock().await; - - tracing::debug!("reschedule plan: {:#?}", reschedule_fragment); - - self.barrier_scheduler - .run_config_change_command_with_pause(Command::RescheduleFragment { - reschedules: reschedule_fragment, - }) - .await?; - - Ok(()) - } - - async fn create_actors_on_compute_node( - &self, - worker_nodes: &HashMap, - actor_infos_to_broadcast: BTreeMap, - node_actors_to_create: HashMap>, - broadcast_worker_ids: HashSet, - ) -> MetaResult<()> { - for worker_id in &broadcast_worker_ids { - let node = worker_nodes.get(worker_id).unwrap(); - let client = self.env.stream_client_pool().get(node).await?; - - let actor_infos_to_broadcast = actor_infos_to_broadcast.values().cloned().collect(); - - client - .to_owned() - .broadcast_actor_info_table(BroadcastActorInfoTableRequest { - info: actor_infos_to_broadcast, - }) - .await?; - } - - for (node_id, stream_actors) in &node_actors_to_create { - let node = worker_nodes.get(node_id).unwrap(); - let client = self.env.stream_client_pool().get(node).await?; - let request_id = Uuid::new_v4().to_string(); - let request = UpdateActorsRequest { - request_id, - actors: stream_actors.clone(), - }; - - client.to_owned().update_actors(request).await?; - } - - for (node_id, stream_actors) in node_actors_to_create { - let node = worker_nodes.get(&node_id).unwrap(); - let client = self.env.stream_client_pool().get(node).await?; - let request_id = Uuid::new_v4().to_string(); - - client - .to_owned() - .build_actors(BuildActorsRequest { - request_id, - actor_id: stream_actors - .iter() - .map(|stream_actor| stream_actor.actor_id) - .collect(), - }) - .await?; - } - - Ok(()) + Ok((reschedule_fragment, applied_reschedules)) } async fn arrange_reschedules( @@ -1593,12 +1494,72 @@ impl GlobalStreamManager { Ok(()) } -} -impl GlobalStreamManager { - async fn generate_stable_resize_plan( + pub async fn post_apply_reschedule( + &self, + reschedules: &HashMap, + ) -> MetaResult>> { + let mut node_dropped_actors = HashMap::new(); + for table_fragments in self + .fragment_manager + .get_fragment_read_guard() + .await + .table_fragments() + .values() + { + for fragment_id in table_fragments.fragments.keys() { + if let Some(reschedule) = reschedules.get(fragment_id) { + for actor_id in &reschedule.removed_actors { + let node_id = table_fragments + .actor_status + .get(actor_id) + .unwrap() + .parallel_unit + .as_ref() + .unwrap() + .worker_node_id; + node_dropped_actors + .entry(node_id as WorkerId) + .or_insert(vec![]) + .push(*actor_id as ActorId); + } + } + } + } + + // Update fragment info after rescheduling in meta store. + self.fragment_manager + .post_apply_reschedules(reschedules.clone()) + .await?; + + let mut stream_source_actor_splits = HashMap::new(); + let mut stream_source_dropped_actors = HashSet::new(); + + for (fragment_id, reschedule) in reschedules { + if !reschedule.actor_splits.is_empty() { + stream_source_actor_splits + .insert(*fragment_id as FragmentId, reschedule.actor_splits.clone()); + stream_source_dropped_actors.extend(reschedule.removed_actors.clone()); + } + } + + if !stream_source_actor_splits.is_empty() { + self.source_manager + .apply_source_change( + None, + Some(stream_source_actor_splits), + Some(stream_source_dropped_actors), + ) + .await; + } + + Ok(node_dropped_actors) + } + + pub async fn generate_stable_resize_plan( &self, policy: StableResizePolicy, + parallel_unit_hints: Option>>, ) -> MetaResult> { let StableResizePolicy { fragment_worker_changes, @@ -1742,7 +1703,12 @@ impl GlobalStreamManager { } for worker_id in include_worker_ids.iter().chain(exclude_worker_ids.iter()) { - if !worker_parallel_units.contains_key(worker_id) { + if !worker_parallel_units.contains_key(worker_id) + && !parallel_unit_hints + .as_ref() + .map(|hints| hints.contains_key(worker_id)) + .unwrap_or(false) + { bail!("Worker id {} not found", worker_id); } } @@ -1759,17 +1725,25 @@ impl GlobalStreamManager { }) .collect(); - let include_worker_parallel_unit_ids = include_worker_ids - .iter() - .flat_map(|worker_id| worker_parallel_units.get(worker_id).unwrap()) - .cloned() - .collect_vec(); + let worker_to_parallel_unit_ids = |worker_ids: &BTreeSet| { + worker_ids + .iter() + .flat_map(|worker_id| { + worker_parallel_units + .get(worker_id) + .or_else(|| { + parallel_unit_hints + .as_ref() + .and_then(|hints| hints.get(worker_id)) + }) + .expect("worker id should be valid") + }) + .cloned() + .collect_vec() + }; - let exclude_worker_parallel_unit_ids = exclude_worker_ids - .iter() - .flat_map(|worker_id| worker_parallel_units.get(worker_id).unwrap()) - .cloned() - .collect_vec(); + let include_worker_parallel_unit_ids = worker_to_parallel_unit_ids(&include_worker_ids); + let exclude_worker_parallel_unit_ids = worker_to_parallel_unit_ids(&exclude_worker_ids); fn refilter_parallel_unit_id_by_target_parallelism( worker_parallel_units: &HashMap>, @@ -1920,7 +1894,101 @@ impl GlobalStreamManager { policy: Policy, ) -> MetaResult> { match policy { - Policy::StableResizePolicy(resize) => self.generate_stable_resize_plan(resize).await, + Policy::StableResizePolicy(resize) => { + self.generate_stable_resize_plan(resize, None).await + } } } + + pub fn build_no_shuffle_relation_index( + actor_map: &HashMap, + no_shuffle_source_fragment_ids: &mut HashSet, + no_shuffle_target_fragment_ids: &mut HashSet, + ) { + for actor in actor_map.values() { + for dispatcher in &actor.dispatcher { + for downstream_actor_id in &dispatcher.downstream_actor_id { + if let Some(downstream_actor) = actor_map.get(downstream_actor_id) { + // Checking for no shuffle dispatchers + if dispatcher.r#type() == DispatcherType::NoShuffle { + no_shuffle_source_fragment_ids.insert(actor.fragment_id as FragmentId); + no_shuffle_target_fragment_ids + .insert(downstream_actor.fragment_id as FragmentId); + } + } + } + } + } + } + + pub fn build_fragment_dispatcher_index( + actor_map: &HashMap, + fragment_dispatcher_map: &mut HashMap>, + ) { + for actor in actor_map.values() { + for dispatcher in &actor.dispatcher { + for downstream_actor_id in &dispatcher.downstream_actor_id { + if let Some(downstream_actor) = actor_map.get(downstream_actor_id) { + fragment_dispatcher_map + .entry(actor.fragment_id as FragmentId) + .or_default() + .insert( + downstream_actor.fragment_id as FragmentId, + dispatcher.r#type(), + ); + } + } + } + } + } + + pub fn resolve_no_shuffle_upstream( + reschedule: &mut HashMap, + fragment_map: &HashMap, + no_shuffle_source_fragment_ids: &HashSet, + no_shuffle_target_fragment_ids: &HashSet, + ) -> MetaResult<()> + where + T: Clone + Eq, + { + let mut queue: VecDeque = reschedule.keys().cloned().collect(); + + // We trace the upstreams of each downstream under the hierarchy until we reach the top + // for every no_shuffle relation. + while let Some(fragment_id) = queue.pop_front() { + if !no_shuffle_target_fragment_ids.contains(&fragment_id) + && !no_shuffle_source_fragment_ids.contains(&fragment_id) + { + continue; + } + + // for upstream + for upstream_fragment_id in &fragment_map + .get(&fragment_id) + .unwrap() + .upstream_fragment_ids + { + if !no_shuffle_source_fragment_ids.contains(upstream_fragment_id) { + continue; + } + + let reschedule_plan = reschedule.get(&fragment_id).unwrap(); + + if let Some(upstream_reschedule_plan) = reschedule.get(upstream_fragment_id) { + if upstream_reschedule_plan != reschedule_plan { + bail!("Inconsistent NO_SHUFFLE plan, check target worker ids of fragment {} and {}", fragment_id, upstream_fragment_id); + } + + continue; + } + + reschedule.insert(*upstream_fragment_id, reschedule_plan.clone()); + queue.push_back(*upstream_fragment_id); + } + } + + reschedule.retain(|fragment_id, _| !no_shuffle_target_fragment_ids.contains(fragment_id)); + + Ok(()) + } } diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 184ca096734e1..acd874f4bdc00 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -29,11 +29,13 @@ use tokio::sync::{oneshot, Mutex, RwLock}; use tracing::Instrument; use uuid::Uuid; -use super::Locations; +use super::{ + Locations, ParallelUnitReschedule, RescheduleOptions, ScaleController, ScaleControllerRef, +}; use crate::barrier::{BarrierScheduler, Command}; use crate::hummock::HummockManagerRef; use crate::manager::{ClusterManagerRef, FragmentManagerRef, MetaSrvEnv}; -use crate::model::{ActorId, TableFragments}; +use crate::model::{ActorId, FragmentId, TableFragments}; use crate::stream::SourceManagerRef; use crate::{MetaError, MetaResult}; @@ -195,6 +197,8 @@ pub struct GlobalStreamManager { hummock_manager: HummockManagerRef, pub reschedule_lock: RwLock<()>, + + pub(crate) scale_controller: ScaleControllerRef, } impl GlobalStreamManager { @@ -206,6 +210,12 @@ impl GlobalStreamManager { source_manager: SourceManagerRef, hummock_manager: HummockManagerRef, ) -> MetaResult { + let scale_controller = Arc::new(ScaleController::new( + fragment_manager.clone(), + cluster_manager.clone(), + source_manager.clone(), + env.clone(), + )); Ok(Self { env, fragment_manager, @@ -215,6 +225,7 @@ impl GlobalStreamManager { hummock_manager, creating_job_info: Arc::new(CreatingStreamingJobInfo::default()), reschedule_lock: RwLock::new(()), + scale_controller, }) } @@ -641,6 +652,61 @@ impl GlobalStreamManager { } } +impl GlobalStreamManager { + pub async fn reschedule_actors( + &self, + reschedules: HashMap, + options: RescheduleOptions, + ) -> MetaResult<()> { + let mut revert_funcs = vec![]; + if let Err(e) = self + .reschedule_actors_impl(&mut revert_funcs, reschedules, options) + .await + { + for revert_func in revert_funcs.into_iter().rev() { + revert_func.await; + } + return Err(e); + } + + Ok(()) + } + + async fn reschedule_actors_impl( + &self, + revert_funcs: &mut Vec>, + reschedules: HashMap, + options: RescheduleOptions, + ) -> MetaResult<()> { + let (reschedule_fragment, applied_reschedules) = self + .scale_controller + .prepare_reschedule_command(reschedules, options) + .await?; + + tracing::debug!("reschedule plan: {:#?}", reschedule_fragment); + + let command = Command::RescheduleFragment { + reschedules: reschedule_fragment, + }; + + let fragment_manager_ref = self.fragment_manager.clone(); + + revert_funcs.push(Box::pin(async move { + fragment_manager_ref + .cancel_apply_reschedules(applied_reschedules) + .await; + })); + + let _source_pause_guard = self.source_manager.paused.lock().await; + + self.barrier_scheduler + .run_config_change_command_with_pause(command) + .await?; + + Ok(()) + } +} + #[cfg(test)] mod tests { use std::collections::{BTreeMap, HashMap, HashSet}; diff --git a/src/tests/simulation/src/cluster.rs b/src/tests/simulation/src/cluster.rs index 202e2f63ed2da..d4947e225008a 100644 --- a/src/tests/simulation/src/cluster.rs +++ b/src/tests/simulation/src/cluster.rs @@ -114,6 +114,27 @@ impl Configuration { } } + pub fn for_auto_scale() -> Self { + let config_path = { + let mut file = + tempfile::NamedTempFile::new().expect("failed to create temp config file"); + file.write_all(include_bytes!("risingwave-auto-scale.toml")) + .expect("failed to write config file"); + file.into_temp_path() + }; + + Configuration { + config_path: ConfigPath::Temp(config_path.into()), + frontend_nodes: 1, + compute_nodes: 3, + meta_nodes: 1, + compactor_nodes: 1, + compute_node_cores: 2, + etcd_timeout_rate: 0.0, + etcd_data_path: None, + } + } + /// Returns the config for backfill test. pub fn for_backfill() -> Self { // Embed the config file and create a temporary file at runtime. The file will be deleted @@ -585,6 +606,23 @@ impl Cluster { .await; } + #[cfg_or_panic(madsim)] + pub async fn kill_nodes_and_restart( + &self, + nodes: impl IntoIterator>, + restart_delay_secs: u32, + ) { + join_all(nodes.into_iter().map(|name| async move { + let name = name.as_ref(); + tracing::info!("kill {name}"); + Handle::current().kill(name); + tokio::time::sleep(Duration::from_secs(restart_delay_secs as u64)).await; + tracing::info!("restart {name}"); + Handle::current().restart(name); + })) + .await; + } + /// Create a node for kafka producer and prepare data. #[cfg_or_panic(madsim)] pub async fn create_kafka_producer(&self, datadir: &str) { diff --git a/src/tests/simulation/src/risingwave-auto-scale.toml b/src/tests/simulation/src/risingwave-auto-scale.toml new file mode 100644 index 0000000000000..f2ffb036df4f2 --- /dev/null +++ b/src/tests/simulation/src/risingwave-auto-scale.toml @@ -0,0 +1,17 @@ +# The configuration for scaling simulation test. +# +# Note: this file is embedded in the binary and cannot be changed without recompiling. + +[meta] +# a relatively small number to make it easier to timeout +max_heartbeat_interval_secs = 15 +meta_leader_lease_secs = 10 +enable_scale_in_when_recovery = true + +[system] +barrier_interval_ms = 250 +checkpoint_frequency = 4 + +[server] +telemetry_enabled = false +metrics_level = "Disabled" diff --git a/src/tests/simulation/tests/integration_tests/recovery/mod.rs b/src/tests/simulation/tests/integration_tests/recovery/mod.rs index 2430daad760a1..d3c5572c8dea0 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/mod.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/mod.rs @@ -16,3 +16,4 @@ mod backfill; mod background_ddl; mod nexmark_recovery; mod pause_on_bootstrap; +mod scale_in_when_recovery; diff --git a/src/tests/simulation/tests/integration_tests/recovery/scale_in_when_recovery.rs b/src/tests/simulation/tests/integration_tests/recovery/scale_in_when_recovery.rs new file mode 100644 index 0000000000000..cfbce605d31d0 --- /dev/null +++ b/src/tests/simulation/tests/integration_tests/recovery/scale_in_when_recovery.rs @@ -0,0 +1,166 @@ +// 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 std::time::Duration; + +use anyhow::Result; +use itertools::Itertools; +use risingwave_pb::common::{WorkerNode, WorkerType}; +use risingwave_simulation::cluster::{Cluster, Configuration}; +use risingwave_simulation::ctl_ext::predicate::{identity_contains, no_identity_contains}; +use risingwave_simulation::utils::AssertResult; +use tokio::time::sleep; + +#[tokio::test] +async fn test_scale_in_when_recovery() -> Result<()> { + let config = Configuration::for_auto_scale(); + let mut cluster = Cluster::start(config.clone()).await?; + let mut session = cluster.start_session(); + + session.run("create table t (v1 int);").await?; + session + .run("create materialized view m as select count(*) from t;") + .await?; + + session + .run("insert into t select * from generate_series(1, 100)") + .await?; + session.run("flush").await?; + + sleep(Duration::from_secs(5)).await; + + let table_mat_fragment = cluster + .locate_one_fragment(vec![ + identity_contains("materialize"), + no_identity_contains("simpleAgg"), + ]) + .await?; + + let single_agg_fragment = cluster + .locate_one_fragment(vec![ + identity_contains("simpleAgg"), + identity_contains("materialize"), + ]) + .await?; + + let (_, single_used_parallel_unit_ids) = single_agg_fragment.parallel_unit_usage(); + + let used_parallel_unit_id = single_used_parallel_unit_ids.iter().next().unwrap(); + + let mut workers: Vec = cluster + .get_cluster_info() + .await? + .worker_nodes + .into_iter() + .filter(|worker| worker.r#type() == WorkerType::ComputeNode) + .collect(); + + let prev_workers = workers + .extract_if(|worker| { + worker + .parallel_units + .iter() + .map(|parallel_unit| parallel_unit.id) + .contains(used_parallel_unit_id) + }) + .collect_vec(); + + let prev_worker = prev_workers.into_iter().exactly_one().unwrap(); + let host = prev_worker.host.unwrap().host; + let host_name = format!("compute-{}", host.split('.').last().unwrap()); + + let (all_parallel_units, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); + + assert_eq!(all_parallel_units.len(), used_parallel_units.len()); + + let initialized_parallelism = used_parallel_units.len(); + + assert_eq!( + initialized_parallelism, + config.compute_nodes * config.compute_node_cores + ); + + // ensure the restart delay is longer than config in `risingwave-auto-scale.toml` + let restart_delay = 30; + + cluster + .kill_nodes_and_restart(vec![host_name], restart_delay) + .await; + + let table_mat_fragment = cluster + .locate_one_fragment(vec![ + identity_contains("materialize"), + no_identity_contains("simpleAgg"), + ]) + .await?; + + let (_, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); + + assert_eq!( + initialized_parallelism - config.compute_node_cores, + used_parallel_units.len() + ); + + let stream_scan_fragment = cluster + .locate_one_fragment(vec![identity_contains("streamTableScan")]) + .await?; + + let (_, used_parallel_units) = stream_scan_fragment.parallel_unit_usage(); + + assert_eq!( + initialized_parallelism - config.compute_node_cores, + used_parallel_units.len() + ); + + let single_agg_fragment = cluster + .locate_one_fragment(vec![ + identity_contains("simpleAgg"), + identity_contains("materialize"), + ]) + .await?; + + let (_, used_parallel_units_ids) = single_agg_fragment.parallel_unit_usage(); + + assert_eq!(used_parallel_units_ids.len(), 1); + + assert_ne!(single_used_parallel_unit_ids, used_parallel_units_ids); + + session + .run("select count(*) from t") + .await? + .assert_result_eq("100"); + + session + .run("select * from m") + .await? + .assert_result_eq("100"); + + session + .run("INSERT INTO t select * from generate_series(101, 150)") + .await?; + + session.run("flush").await?; + + session + .run("select count(*) from t") + .await? + .assert_result_eq("150"); + + session + .run("select * from m") + .await? + .assert_result_eq("150"); + + Ok(()) +} From e0ff37f3b2b50ee585715a3719dcb78924f01bd2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 9 Nov 2023 02:36:15 +0000 Subject: [PATCH 18/77] chore(deps): Bump mysql_async from 0.32.2 to 0.33.0 (#13323) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: xxchan --- .github/dependabot.yml | 4 ++ Cargo.lock | 101 ++++++++++++++++++++++----------------- src/connector/Cargo.toml | 4 +- 3 files changed, 63 insertions(+), 46 deletions(-) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 51242d0425e28..d0cf53d2b2f8d 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -24,3 +24,7 @@ updates: patterns: - "tonic*" - "prost*" + mysql: + patterns: + - "mysql_common" + - "mysql_async" diff --git a/Cargo.lock b/Cargo.lock index 8fd4e50b5e139..d68628a608438 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1232,6 +1232,19 @@ dependencies = [ "num-traits", ] +[[package]] +name = "bigdecimal" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c06619be423ea5bb86c95f087d5707942791a08a85530df0db2209a3ecfb8bc9" +dependencies = [ + "autocfg", + "libm", + "num-bigint", + "num-integer", + "num-traits", +] + [[package]] name = "bincode" version = "1.3.3" @@ -1436,6 +1449,15 @@ dependencies = [ "memchr", ] +[[package]] +name = "btoi" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9dd6407f73a9b8b6162d8a2ef999fe6afd7cc15902ebf42c5cd296addf17e0ad" +dependencies = [ + "num-traits", +] + [[package]] name = "bumpalo" version = "3.13.0" @@ -4286,6 +4308,15 @@ dependencies = [ "simple_asn1", ] +[[package]] +name = "keyed_priority_queue" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2d63b6407b66fc81fc539dccf3ddecb669f393c5101b6a2be3976c95099a06e8" +dependencies = [ + "indexmap 1.9.3", +] + [[package]] name = "krb5-src" version = "0.3.2+1.19.2" @@ -4319,15 +4350,6 @@ version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" -[[package]] -name = "lexical" -version = "6.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7aefb36fd43fef7003334742cbf77b243fcd36418a1d1bdd480d613a67968f6" -dependencies = [ - "lexical-core", -] - [[package]] name = "lexical-core" version = "0.8.5" @@ -4560,6 +4582,15 @@ dependencies = [ "hashbrown 0.13.2", ] +[[package]] +name = "lru" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1efa59af2ddfad1854ae27d75009d538d0998b4b2fd47083e743ac1a10e46c60" +dependencies = [ + "hashbrown 0.14.0", +] + [[package]] name = "lz4" version = "1.24.0" @@ -4978,9 +5009,9 @@ dependencies = [ [[package]] name = "mysql_async" -version = "0.32.2" +version = "0.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5272f59b5b1f93d65f7f826c1f025d6e410e89fb50a67e05aa20b35a55a8c0a" +checksum = "6750b17ce50f8f112ef1a8394121090d47c596b56a6a17569ca680a9626e2ef2" dependencies = [ "bytes", "crossbeam", @@ -4988,16 +5019,17 @@ dependencies = [ "futures-core", "futures-sink", "futures-util", + "keyed_priority_queue", "lazy_static", - "lru 0.10.1", + "lru 0.12.0", "mio", "mysql_common", "native-tls", "once_cell", - "pem 2.0.1", + "pem 3.0.2", "percent-encoding", "pin-project", - "priority-queue", + "rand", "serde", "serde_json", "socket2 0.5.3", @@ -5011,15 +5043,16 @@ dependencies = [ [[package]] name = "mysql_common" -version = "0.30.6" +version = "0.31.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57349d5a326b437989b6ee4dc8f2f34b0cc131202748414712a8e7d98952fc8c" +checksum = "06f19e4cfa0ab5a76b627cec2d81331c49b034988eaf302c3bafeada684eadef" dependencies = [ "base64 0.21.4", - "bigdecimal", + "bigdecimal 0.4.2", "bindgen", "bitflags 2.4.0", "bitvec", + "btoi", "byteorder", "bytes", "cc", @@ -5029,7 +5062,6 @@ dependencies = [ "flate2", "frunk", "lazy_static", - "lexical", "mysql-common-derive", "num-bigint", "num-traits", @@ -5046,6 +5078,7 @@ dependencies = [ "thiserror", "time", "uuid", + "zstd 0.12.4", ] [[package]] @@ -5882,16 +5915,6 @@ dependencies = [ "base64 0.13.1", ] -[[package]] -name = "pem" -version = "2.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b13fe415cdf3c8e44518e18a7c95a13431d9bdf6d15367d82b23c377fdd441a" -dependencies = [ - "base64 0.21.4", - "serde", -] - [[package]] name = "pem" version = "3.0.2" @@ -6298,16 +6321,6 @@ dependencies = [ "elliptic-curve", ] -[[package]] -name = "priority-queue" -version = "1.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fff39edfcaec0d64e8d0da38564fad195d2d51b680940295fcc307366e101e61" -dependencies = [ - "autocfg", - "indexmap 1.9.3", -] - [[package]] name = "proc-macro-crate" version = "0.1.5" @@ -8779,7 +8792,7 @@ checksum = "61f6c7daef05dde3476d97001e11fca7a52b655aa3bf4fd610ab2da1176a2ed5" dependencies = [ "async-stream", "async-trait", - "bigdecimal", + "bigdecimal 0.3.1", "chrono", "futures", "log", @@ -8853,7 +8866,7 @@ version = "0.30.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "28c05a5bf6403834be253489bbe95fa9b1e5486bc843b61f60d26b5c9c1e244b" dependencies = [ - "bigdecimal", + "bigdecimal 0.3.1", "chrono", "derivative", "inherent", @@ -8871,7 +8884,7 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "36bbb68df92e820e4d5aeb17b4acd5cc8b5d18b2c36a4dd6f4626aabfa7ab1b9" dependencies = [ - "bigdecimal", + "bigdecimal 0.3.1", "chrono", "rust_decimal", "sea-query", @@ -9531,7 +9544,7 @@ checksum = "dd4cef4251aabbae751a3710927945901ee1d97ee96d757f6880ebb9a79bfd53" dependencies = [ "ahash 0.8.3", "atoi", - "bigdecimal", + "bigdecimal 0.3.1", "byteorder", "bytes", "chrono", @@ -9616,7 +9629,7 @@ checksum = "8ca69bf415b93b60b80dc8fda3cb4ef52b2336614d8da2de5456cc942a110482" dependencies = [ "atoi", "base64 0.21.4", - "bigdecimal", + "bigdecimal 0.3.1", "bitflags 2.4.0", "byteorder", "bytes", @@ -9663,7 +9676,7 @@ checksum = "a0db2df1b8731c3651e204629dd55e52adbae0462fa1bdcbed56a2302c18181e" dependencies = [ "atoi", "base64 0.21.4", - "bigdecimal", + "bigdecimal 0.3.1", "bitflags 2.4.0", "byteorder", "chrono", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 50e8f5fd9b708..fe31e1a5c6bc4 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -68,10 +68,10 @@ jni = { version = "0.21.1", features = ["invocation"] } jst = { package = 'jsonschema-transpiler', git = "https://github.com/mozilla/jsonschema-transpiler", rev = "c1a89d720d118843d8bcca51084deb0ed223e4b4" } maplit = "1.0.2" moka = { version = "0.12", features = ["future"] } -mysql_async = { version = "0.32", default-features = false, features = [ +mysql_async = { version = "0.33", default-features = false, features = [ "default", ] } -mysql_common = { version = "0.30", default-features = false, features = [ +mysql_common = { version = "0.31", default-features = false, features = [ "chrono", ] } nexmark = { version = "0.2", features = ["serde"] } From eb4218cc9c0d9d5a14e358d61dd88d26063597c2 Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 9 Nov 2023 13:20:10 +0800 Subject: [PATCH 19/77] fix(storage): Handle heartbeat cancel in fast compactor runner (#13315) --- src/meta/src/hummock/manager/mod.rs | 5 +-- src/storage/hummock_sdk/src/compact.rs | 6 ++++ .../src/hummock/compactor/compactor_runner.rs | 34 ++++++++++++------- 3 files changed, 31 insertions(+), 14 deletions(-) diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 13a9906de9ac0..47c27d758d79e 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -2748,9 +2748,10 @@ impl HummockManager { // TODO: task cancellation can be batched for task in cancel_tasks { tracing::info!( - "Task with task_id {} with context_id {} has expired due to lack of visible progress", + "Task with group_id {} task_id {} with context_id {} has expired due to lack of visible progress", + task.compaction_group_id, + task.task_id, context_id, - task.task_id ); if let Err(e) = diff --git a/src/storage/hummock_sdk/src/compact.rs b/src/storage/hummock_sdk/src/compact.rs index bb01035b88442..4d590a558d79d 100644 --- a/src/storage/hummock_sdk/src/compact.rs +++ b/src/storage/hummock_sdk/src/compact.rs @@ -41,6 +41,12 @@ pub fn compact_task_to_string(compact_task: &CompactTask) -> String { compact_task.task_status() ) .unwrap(); + writeln!( + s, + "Compaction task table_ids: {:?} ", + compact_task.existing_table_ids, + ) + .unwrap(); s.push_str("Compaction Sstables structure: \n"); for level_entry in &compact_task.input_ssts { let tables: Vec = level_entry diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 67b18ce392429..1fcd519179e40 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -472,17 +472,27 @@ pub async fn compact( object_id_getter.clone(), task_progress_guard.progress.clone(), ); - match runner.run().await { - Ok((ssts, statistics)) => { - output_ssts.push((0, ssts, statistics)); - } - Err(e) => { - task_status = TaskStatus::ExecuteFailed; - tracing::warn!( - "Compaction task {} failed with error: {:#?}", - compact_task.task_id, - e - ); + + tokio::select! { + _ = &mut shutdown_rx => { + tracing::warn!("Compaction task cancelled externally:\n{}", compact_task_to_string(&compact_task)); + task_status = TaskStatus::ManualCanceled; + }, + + ret = runner.run() => { + match ret { + Ok((ssts, statistics)) => { + output_ssts.push((0, ssts, statistics)); + } + Err(e) => { + task_status = TaskStatus::ExecuteFailed; + tracing::warn!( + "Compaction task {} failed with error: {:#?}", + compact_task.task_id, + e + ); + } + } } } @@ -492,7 +502,7 @@ pub async fn compact( compact_done(compact_task, context.clone(), output_ssts, task_status); let cost_time = timer.stop_and_record() * 1000.0; tracing::info!( - "Finished compaction task in {:?}ms: {}", + "Finished fast compaction task in {:?}ms: {}", cost_time, compact_task_to_string(&compact_task) ); From 33bc6b7ef8872f8ddca5ab280ef82ca0a03711d8 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 9 Nov 2023 15:11:20 +0800 Subject: [PATCH 20/77] docs: add quick reference for ci labels (#13335) --- docs/developer-guide.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/docs/developer-guide.md b/docs/developer-guide.md index 7d072e7da2e44..2782db7e29e62 100644 --- a/docs/developer-guide.md +++ b/docs/developer-guide.md @@ -47,6 +47,7 @@ http://ecotrust-canada.github.io/markdown-toc/ - [Submit PRs](#submit-prs) - [Profiling](#benchmarking-and-profiling) - [Understanding RisingWave Macros](#understanding-risingwave-macros) +- [CI Labels Guide](#ci-labels-guide) ## Read the design docs @@ -523,3 +524,18 @@ Instructions about submitting PRs are included in the [contribution guidelines]( - [CPU Profiling Guide](./cpu-profiling.md) - [Memory (Heap) Profiling Guide](./memory-profiling.md) - [Microbench Guide](./microbenchmarks.md) + +## CI Labels Guide + +- `ci/skip-ci` + `[ci/run-xxx ...]` : Run specific steps indicated by `ci/run-xxx` in your **DRAFT PR.** +- `ci/run-main-cron`: Run full `main-cron`. +- `ci/run-main-cron` + `ci/main-cron/skip-ci` + `[ci/run-xxx …]` : Run specific steps indicated by `ci/run-xxx` + from the `main-cron` workflow, in your PR. Can use to verify some `main-cron` fix works as expected. +- **Be sure to add all the dependencies.** + + For example to run `e2e-test` for `main-cron` in your pull request: + 1. Add `ci/run-build`, `ci/run-build-other`, `ci/run-docslt` . + These correspond to its `depends` field in `pull-request.yml` and `main-cron.yml` . + 2. Add `ci/run-e2e-test` to run the step as well. + 3. Add `ci/run-main-cron` to run `main-cron` workflow in your pull request, + 4. Add `ci/main-cron/skip-ci` to skip all other steps which were not selected with `ci/run-xxx`. \ No newline at end of file From b7b6c6c65aa14cfb925c50e23b345d440dd6beec Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 9 Nov 2023 15:17:08 +0800 Subject: [PATCH 21/77] refactor: use `expect-test` to generate default config file (#13333) Signed-off-by: Bugen Zhao --- Cargo.lock | 1 + Makefile.toml | 6 +----- src/common/Cargo.toml | 5 +---- src/common/src/bin/default_config.rs | 21 --------------------- src/common/src/config.rs | 20 ++++++++------------ src/config/example.toml | 1 + src/connector/src/lib.rs | 1 - 7 files changed, 12 insertions(+), 43 deletions(-) delete mode 100644 src/common/src/bin/default_config.rs diff --git a/Cargo.lock b/Cargo.lock index d68628a608438..2726498ea9856 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7283,6 +7283,7 @@ dependencies = [ "enum-as-inner", "enumflags2", "ethnum", + "expect-test", "fixedbitset", "fs-err", "futures", diff --git a/Makefile.toml b/Makefile.toml index 6620972028653..c60da632cd6aa 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -1430,11 +1430,7 @@ private = true category = "Misc" description = "Generate default config and write to src/config/example.toml" script = ''' -cat << EOF > src/config/example.toml -# This file is generated by ./risedev generate-example-config -# Check detailed comments in src/common/src/config.rs -EOF -cargo run -p risingwave_common --bin example-config >> src/config/example.toml +UPDATE_EXPECT=1 cargo test --package risingwave_common --lib -- config::tests::test_example_up_to_date ''' [tasks.generate-with-options] diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 70463d8ea205a..1f8787c153a85 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -123,6 +123,7 @@ mach2 = "0.4" [dev-dependencies] criterion = { workspace = true } +expect-test = "1" more-asserts = "0.3" pretty_assertions = "1" rand = "0.8" @@ -153,9 +154,5 @@ harness = false name = "bench_array" harness = false -[[bin]] -name = "example-config" -path = "src/bin/default_config.rs" - [lints] workspace = true diff --git a/src/common/src/bin/default_config.rs b/src/common/src/bin/default_config.rs deleted file mode 100644 index 98f66854d2d04..0000000000000 --- a/src/common/src/bin/default_config.rs +++ /dev/null @@ -1,21 +0,0 @@ -// 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_common::config::RwConfig; - -fn main() { - let config = RwConfig::default(); - let toml = toml::to_string(&config).unwrap(); - print!("{}", &toml); -} diff --git a/src/common/src/config.rs b/src/common/src/config.rs index bba8f50d5af68..02b97bf6e618c 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -1482,17 +1482,13 @@ mod tests { /// test fails. #[test] fn test_example_up_to_date() { - let actual = { - let content = include_str!("../../config/example.toml"); - toml::from_str::(content).expect("parse example.toml failed") - }; - let expected = - toml::Value::try_from(RwConfig::default()).expect("serialize default config failed"); - - // Compare the `Value` representation instead of string for normalization. - pretty_assertions::assert_eq!( - actual, expected, - "\n`config/example.toml` is not up-to-date with the default values specified in `config.rs`.\nPlease run `./risedev generate-example-config` to update it." - ); + const HEADER: &str = "# This file is generated by ./risedev generate-example-config +# Check detailed comments in src/common/src/config.rs"; + + let actual = expect_test::expect_file!["../../config/example.toml"]; + let default = toml::to_string(&RwConfig::default()).expect("failed to serialize"); + + let expected = format!("{HEADER}\n\n{default}"); + actual.assert_eq(&expected); } } diff --git a/src/config/example.toml b/src/config/example.toml index 61d9e23544a05..8e8a2ac0bb406 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -1,5 +1,6 @@ # This file is generated by ./risedev generate-example-config # Check detailed comments in src/common/src/config.rs + [server] heartbeat_interval_ms = 1000 connection_pool_size = 16 diff --git a/src/connector/src/lib.rs b/src/connector/src/lib.rs index 88fd75c24ff9a..a57d20b70c626 100644 --- a/src/connector/src/lib.rs +++ b/src/connector/src/lib.rs @@ -131,7 +131,6 @@ mod tests { let expected = expect_file!("../with_options.yaml"); let actual = update_with_options_yaml(); - // Compare the `Value` representation instead of string for normalization. expected.assert_eq(&actual); } } From ff960a50c955bc1255359e70a68c440bcde90abd Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Thu, 9 Nov 2023 16:14:38 +0800 Subject: [PATCH 22/77] chore(proto): standardize name of enums (#13336) --- .../com/risingwave/java/utils/MetaClient.java | 2 +- .../sink/iceberg/IcebergSinkFactoryTest.java | 4 +-- .../connector/CassandraFactory.java | 6 ++-- .../connector/DeltaLakeSinkFactory.java | 3 +- .../connector/IcebergSinkFactory.java | 6 ++-- .../risingwave/connector/JDBCSinkFactory.java | 2 +- java/udf-example/pom.xml | 1 + proto/catalog.proto | 30 +++++++++---------- proto/common.proto | 12 ++++---- proto/ddl_service.proto | 6 ++-- proto/hummock.proto | 6 ++-- proto/plan_common.proto | 18 +++++------ proto/stream_plan.proto | 28 ++++++++--------- src/common/src/catalog/mod.rs | 7 +++-- .../src/parser/debezium/avro_parser.rs | 2 +- .../src/schema/schema_registry/mod.rs | 4 +-- src/frontend/src/handler/create_source.rs | 11 ++++--- .../optimizer/plan_node/stream_table_scan.rs | 2 +- src/stream/src/from_proto/stream_scan.rs | 2 +- 19 files changed, 77 insertions(+), 75 deletions(-) diff --git a/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java index 1406164daed12..87789512441f3 100644 --- a/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java +++ b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java @@ -93,7 +93,7 @@ public MetaClient(String metaAddr, ScheduledExecutorService scheduler) { AddWorkerNodeRequest req = AddWorkerNodeRequest.newBuilder() - .setWorkerType(WorkerType.RISE_CTL) + .setWorkerType(WorkerType.WORKER_TYPE_RISE_CTL) .setHost( HostAddress.newBuilder().setHost("127.0.0.1").setPort(8880).build()) .setProperty( diff --git a/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/iceberg/IcebergSinkFactoryTest.java b/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/iceberg/IcebergSinkFactoryTest.java index a75de39491753..ae620bb018915 100755 --- a/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/iceberg/IcebergSinkFactoryTest.java +++ b/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/sink/iceberg/IcebergSinkFactoryTest.java @@ -117,7 +117,7 @@ public void testValidateSchemaName() throws IOException { .setTypeName(Data.DataType.TypeName.VARCHAR) .build()), Lists.newArrayList("id")); - sinkFactory.validate(diffTypeTableSchema, tableProperties, SinkType.APPEND_ONLY); + sinkFactory.validate(diffTypeTableSchema, tableProperties, SinkType.SINK_TYPE_APPEND_ONLY); } @Test(expected = RuntimeException.class) @@ -145,6 +145,6 @@ public void testValidateSchemaType() throws IOException { .setTypeName(Data.DataType.TypeName.INT32) .build()), Lists.newArrayList("id")); - sinkFactory.validate(diffTypeTableSchema, tableProperties, SinkType.APPEND_ONLY); + sinkFactory.validate(diffTypeTableSchema, tableProperties, SinkType.SINK_TYPE_APPEND_ONLY); } } diff --git a/java/connector-node/risingwave-sink-cassandra/src/main/java/com/risingwave/connector/CassandraFactory.java b/java/connector-node/risingwave-sink-cassandra/src/main/java/com/risingwave/connector/CassandraFactory.java index f9fb5bee020e2..8881aff2bfef1 100644 --- a/java/connector-node/risingwave-sink-cassandra/src/main/java/com/risingwave/connector/CassandraFactory.java +++ b/java/connector-node/risingwave-sink-cassandra/src/main/java/com/risingwave/connector/CassandraFactory.java @@ -85,12 +85,12 @@ public void validate( // 3. close client session.close(); switch (sinkType) { - case UPSERT: + case SINK_TYPE_UPSERT: CassandraUtil.checkPrimaryKey( tableMetadata.getPrimaryKey(), tableSchema.getPrimaryKeys()); break; - case APPEND_ONLY: - case FORCE_APPEND_ONLY: + case SINK_TYPE_APPEND_ONLY: + case SINK_TYPE_FORCE_APPEND_ONLY: break; default: throw Status.INTERNAL.asRuntimeException(); diff --git a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java index 2f683f46bb756..7487189332a39 100644 --- a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java +++ b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java @@ -49,7 +49,8 @@ public SinkWriter createWriter(TableSchema tableSchema, Map tabl @Override public void validate( TableSchema tableSchema, Map tableProperties, SinkType sinkType) { - if (sinkType != SinkType.APPEND_ONLY && sinkType != SinkType.FORCE_APPEND_ONLY) { + if (sinkType != SinkType.SINK_TYPE_APPEND_ONLY + && sinkType != SinkType.SINK_TYPE_FORCE_APPEND_ONLY) { throw Status.INVALID_ARGUMENT .withDescription("only append-only delta lake sink is supported") .asRuntimeException(); diff --git a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java index be568dc85939e..fe96d8591b743 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java +++ b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java @@ -144,7 +144,7 @@ public void validate( } switch (sinkType) { - case UPSERT: + case SINK_TYPE_UPSERT: // For upsert iceberg sink, the user must specify its primary key explicitly. if (tableSchema.getPrimaryKeys().isEmpty()) { throw Status.INVALID_ARGUMENT @@ -152,8 +152,8 @@ public void validate( .asRuntimeException(); } break; - case APPEND_ONLY: - case FORCE_APPEND_ONLY: + case SINK_TYPE_APPEND_ONLY: + case SINK_TYPE_FORCE_APPEND_ONLY: break; default: throw Status.INTERNAL.asRuntimeException(); diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java index e4d009d32b1a0..0a25dd8d1d1dc 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java @@ -96,7 +96,7 @@ public void validate( } } - if (sinkType == SinkType.UPSERT) { + if (sinkType == SinkType.SINK_TYPE_UPSERT) { // For upsert JDBC sink, the primary key defined on the table must match the one in // config and cannot be empty var pkInWith = new HashSet<>(tableSchema.getPrimaryKeys()); diff --git a/java/udf-example/pom.xml b/java/udf-example/pom.xml index dd3e54aca1fa2..c71022ccd2987 100644 --- a/java/udf-example/pom.xml +++ b/java/udf-example/pom.xml @@ -24,6 +24,7 @@ UTF-8 11 11 + true diff --git a/proto/catalog.proto b/proto/catalog.proto index 1372e93c2d2e1..99e3e7539443e 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -27,24 +27,24 @@ message WatermarkDesc { } enum SchemaRegistryNameStrategy { - TOPIC_NAME_STRATEGY_UNSPECIFIED = 0; - RECORD_NAME_STRATEGY = 1; - TOPIC_RECORD_NAME_STRATEGY = 2; + SCHEMA_REGISTRY_NAME_STRATEGY_UNSPECIFIED = 0; + SCHEMA_REGISTRY_NAME_STRATEGY_RECORD_NAME_STRATEGY = 1; + SCHEMA_REGISTRY_NAME_STRATEGY_TOPIC_RECORD_NAME_STRATEGY = 2; } enum StreamJobStatus { // Prefixed by `STREAM_JOB_STATUS` due to protobuf namespacing rules. STREAM_JOB_STATUS_UNSPECIFIED = 0; - CREATING = 1; - CREATED = 2; + STREAM_JOB_STATUS_CREATING = 1; + STREAM_JOB_STATUS_CREATED = 2; } // How the stream job was created will determine // whether they are persisted. enum CreateType { CREATE_TYPE_UNSPECIFIED = 0; - BACKGROUND = 1; - FOREGROUND = 2; + CREATE_TYPE_BACKGROUND = 1; + CREATE_TYPE_FOREGROUND = 2; } message StreamSourceInfo { @@ -104,10 +104,10 @@ message Source { } enum SinkType { - UNSPECIFIED = 0; - APPEND_ONLY = 1; - FORCE_APPEND_ONLY = 2; - UPSERT = 3; + SINK_TYPE_UNSPECIFIED = 0; + SINK_TYPE_APPEND_ONLY = 1; + SINK_TYPE_FORCE_APPEND_ONLY = 2; + SINK_TYPE_UPSERT = 3; } // Similar to `StreamSourceInfo`, and may replace `SinkType` later. @@ -293,10 +293,10 @@ message Table { } enum HandleConflictBehavior { - CONFLICT_BEHAVIOR_UNSPECIFIED = 0; - OVERWRITE = 1; - IGNORE = 2; - NO_CHECK = 3; + HANDLE_CONFLICT_BEHAVIOR_UNSPECIFIED = 0; + HANDLE_CONFLICT_BEHAVIOR_OVERWRITE = 1; + HANDLE_CONFLICT_BEHAVIOR_IGNORE = 2; + HANDLE_CONFLICT_BEHAVIOR_NO_CHECK = 3; } message View { diff --git a/proto/common.proto b/proto/common.proto index 8a4b7006217bf..fe976fb5c8e87 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -27,12 +27,12 @@ message ActorInfo { } enum WorkerType { - UNSPECIFIED = 0; - FRONTEND = 1; - COMPUTE_NODE = 2; - RISE_CTL = 3; - COMPACTOR = 4; - META = 5; + WORKER_TYPE_UNSPECIFIED = 0; + WORKER_TYPE_FRONTEND = 1; + WORKER_TYPE_COMPUTE_NODE = 2; + WORKER_TYPE_RISE_CTL = 3; + WORKER_TYPE_COMPACTOR = 4; + WORKER_TYPE_META = 5; } message ParallelUnit { diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index aaca4f332faae..7cc76234acdb6 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -144,11 +144,11 @@ message DropViewResponse { // - SHARED_CDC_SOURCE: The table streaming job is created based on a shared CDC source job (risingwavelabs/rfcs#73). // And one may add other types to support Table jobs that based on other backfill-able sources (risingwavelabs/rfcs#72). enum TableJobType { - UNSPECIFIED = 0; + TABLE_JOB_TYPE_UNSPECIFIED = 0; // table streaming jobs excepts the `SHARED_CDC_SOURCE` type - GENERAL = 1; + TABLE_JOB_TYPE_GENERAL = 1; // table streaming job sharing a CDC source job - SHARED_CDC_SOURCE = 2; + TABLE_JOB_TYPE_SHARED_CDC_SOURCE = 2; } message CreateTableRequest { diff --git a/proto/hummock.proto b/proto/hummock.proto index ee7d532974ff3..f51d568a2363d 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -31,9 +31,9 @@ message SstableInfo { } enum LevelType { - UNSPECIFIED = 0; - NONOVERLAPPING = 1; - OVERLAPPING = 2; + LEVEL_TYPE_UNSPECIFIED = 0; + LEVEL_TYPE_NONOVERLAPPING = 1; + LEVEL_TYPE_OVERLAPPING = 2; } message OverlappingLevel { diff --git a/proto/plan_common.proto b/proto/plan_common.proto index ad9d58e3b2cc1..9f43d9755edfe 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -89,15 +89,15 @@ message ExternalTableDesc { enum JoinType { // Note that it comes from Calcite's JoinRelType. // DO NOT HAVE direction for SEMI and ANTI now. - UNSPECIFIED = 0; - INNER = 1; - LEFT_OUTER = 2; - RIGHT_OUTER = 3; - FULL_OUTER = 4; - LEFT_SEMI = 5; - LEFT_ANTI = 6; - RIGHT_SEMI = 7; - RIGHT_ANTI = 8; + JOIN_TYPE_UNSPECIFIED = 0; + JOIN_TYPE_INNER = 1; + JOIN_TYPE_LEFT_OUTER = 2; + JOIN_TYPE_RIGHT_OUTER = 3; + JOIN_TYPE_FULL_OUTER = 4; + JOIN_TYPE_LEFT_SEMI = 5; + JOIN_TYPE_LEFT_ANTI = 6; + JOIN_TYPE_RIGHT_SEMI = 7; + JOIN_TYPE_RIGHT_ANTI = 8; } // https://github.com/tokio-rs/prost/issues/80 diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 86b1b89d079ec..2cd8380e3bcf2 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -205,8 +205,8 @@ message SinkDesc { enum SinkLogStoreType { /// Default value is the normal in memory log store to be backward compatible with the previously unset value SINK_LOG_STORE_TYPE_UNSPECIFIED = 0; - KV_LOG_STORE = 1; - IN_MEMORY_LOG_STORE = 2; + SINK_LOG_STORE_TYPE_KV_LOG_STORE = 1; + SINK_LOG_STORE_TYPE_IN_MEMORY_LOG_STORE = 2; } message SinkNode { @@ -449,22 +449,22 @@ message ExchangeNode { // Decides which kind of Executor will be used enum StreamScanType { - CHAIN_UNSPECIFIED = 0; + STREAM_SCAN_TYPE_UNSPECIFIED = 0; // ChainExecutor - CHAIN = 1; + STREAM_SCAN_TYPE_CHAIN = 1; // RearrangedChainExecutor - REARRANGE = 2; + STREAM_SCAN_TYPE_REARRANGE = 2; // BackfillExecutor - BACKFILL = 3; + STREAM_SCAN_TYPE_BACKFILL = 3; // ChainExecutor with upstream_only = true - UPSTREAM_ONLY = 4; + STREAM_SCAN_TYPE_UPSTREAM_ONLY = 4; // CdcBackfillExecutor - CDC_BACKFILL = 5; + STREAM_SCAN_TYPE_CDC_BACKFILL = 5; } // StreamScanNode reads data from upstream table first, and then pass all events to downstream. @@ -713,25 +713,25 @@ message StreamNode { } enum DispatcherType { - UNSPECIFIED = 0; + DISPATCHER_TYPE_UNSPECIFIED = 0; // Dispatch by hash key, hashed by consistent hash. - HASH = 1; + DISPATCHER_TYPE_HASH = 1; // Broadcast to all downstreams. // // Note a broadcast cannot be represented as multiple simple dispatchers, since they are // different when we update dispatchers during scaling. - BROADCAST = 2; + DISPATCHER_TYPE_BROADCAST = 2; // Only one downstream. - SIMPLE = 3; + DISPATCHER_TYPE_SIMPLE = 3; // A special kind of exchange that doesn't involve shuffle. The upstream actor will be directly // piped into the downstream actor, if there are the same number of actors. If number of actors // are not the same, should use hash instead. Should be only used when distribution is the same. - NO_SHUFFLE = 4; + DISPATCHER_TYPE_NO_SHUFFLE = 4; // Dispatch by table name from upstream DB, used in CDC scenario which should has only one downstream actor. // From the optimizer's point of view, it can be treated as a specialized version of HASH distribution // that the hash key is the upstream table name. - CDC_TABLENAME = 5; + DISPATCHER_TYPE_CDC_TABLENAME = 5; } // The property of an edge in the fragment graph. diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 888ca5c2d4067..aa66b0947362d 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -483,9 +483,10 @@ impl ConflictBehavior { PbHandleConflictBehavior::Overwrite => ConflictBehavior::Overwrite, PbHandleConflictBehavior::Ignore => ConflictBehavior::IgnoreConflict, // This is for backward compatibility, in the previous version - // `ConflictBehaviorUnspecified' represented `NoCheck`, so just treat it as `NoCheck`. - PbHandleConflictBehavior::NoCheck - | PbHandleConflictBehavior::ConflictBehaviorUnspecified => ConflictBehavior::NoCheck, + // `HandleConflictBehavior::Unspecified` represented `NoCheck`, so just treat it as `NoCheck`. + PbHandleConflictBehavior::NoCheck | PbHandleConflictBehavior::Unspecified => { + ConflictBehavior::NoCheck + } } } diff --git a/src/connector/src/parser/debezium/avro_parser.rs b/src/connector/src/parser/debezium/avro_parser.rs index f37b4a16d60eb..e18560f6c747b 100644 --- a/src/connector/src/parser/debezium/avro_parser.rs +++ b/src/connector/src/parser/debezium/avro_parser.rs @@ -113,7 +113,7 @@ impl DebeziumAvroParserConfig { let client = Client::new(url, client_config)?; let resolver = ConfluentSchemaResolver::new(client); - let name_strategy = &PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified; + let name_strategy = &PbSchemaRegistryNameStrategy::Unspecified; let key_subject = get_subject_by_strategy(name_strategy, kafka_topic, None, true)?; let val_subject = get_subject_by_strategy(name_strategy, kafka_topic, None, false)?; let key_schema = resolver.get_by_subject_name(&key_subject).await?; diff --git a/src/connector/src/schema/schema_registry/mod.rs b/src/connector/src/schema/schema_registry/mod.rs index 85821f59b8115..0663e8a9be1d0 100644 --- a/src/connector/src/schema/schema_registry/mod.rs +++ b/src/connector/src/schema/schema_registry/mod.rs @@ -22,7 +22,7 @@ pub(crate) use util::*; pub fn name_strategy_from_str(value: &str) -> Option { match value { - "topic_name_strategy" => Some(PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified), + "topic_name_strategy" => Some(PbSchemaRegistryNameStrategy::Unspecified), "record_name_strategy" => Some(PbSchemaRegistryNameStrategy::RecordNameStrategy), "topic_record_name_strategy" => Some(PbSchemaRegistryNameStrategy::TopicRecordNameStrategy), _ => None, @@ -44,7 +44,7 @@ pub fn get_subject_by_strategy( }; let record_option_name = if is_key { "key.message" } else { "message" }; match name_strategy { - PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified => { + PbSchemaRegistryNameStrategy::Unspecified => { // default behavior let suffix = if is_key { "key" } else { "value" }; Ok(format!("{topic}-{suffix}",)) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 637b2b544f67b..f7aa512b8bef7 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -350,9 +350,8 @@ pub(crate) async fn bind_columns_from_source( use_schema_registry: protobuf_schema.use_schema_registry, proto_message_name: protobuf_schema.message_name.0.clone(), key_message_name: get_key_message_name(&mut options), - name_strategy: name_strategy.unwrap_or( - PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified as i32, - ), + name_strategy: name_strategy + .unwrap_or(PbSchemaRegistryNameStrategy::Unspecified as i32), ..Default::default() }, ) @@ -395,7 +394,7 @@ pub(crate) async fn bind_columns_from_source( proto_message_name: message_name.unwrap_or(AstString("".into())).0, key_message_name, name_strategy: name_strategy - .unwrap_or(PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified as i32), + .unwrap_or(PbSchemaRegistryNameStrategy::Unspecified as i32), ..Default::default() }; ( @@ -459,7 +458,7 @@ pub(crate) async fn bind_columns_from_source( let name_strategy = get_sr_name_strategy_check(&mut options, avro_schema.use_schema_registry)? - .unwrap_or(PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified as i32); + .unwrap_or(PbSchemaRegistryNameStrategy::Unspecified as i32); let key_message_name = get_key_message_name(&mut options); let message_name = try_consume_string_from_options(&mut options, MESSAGE_NAME_KEY); @@ -511,7 +510,7 @@ pub(crate) async fn bind_columns_from_source( use_schema_registry, proto_message_name: message_name.unwrap_or(AstString("".into())).0, name_strategy: name_strategy - .unwrap_or(PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified as i32), + .unwrap_or(PbSchemaRegistryNameStrategy::Unspecified as i32), format: FormatType::Debezium as i32, row_encode: EncodeType::Avro as i32, row_schema_location: avro_schema.row_schema_location.0.clone(), diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 0fa89b87e1848..1b6aa8d84aa14 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -254,7 +254,7 @@ impl StreamTableScan { StreamScanType::Chain | StreamScanType::Rearrange | StreamScanType::UpstreamOnly => { self.core.output_column_ids() } - StreamScanType::ChainUnspecified => unreachable!(), + StreamScanType::Unspecified => unreachable!(), } .iter() .map(ColumnId::get_id) diff --git a/src/stream/src/from_proto/stream_scan.rs b/src/stream/src/from_proto/stream_scan.rs index 6b6247e9509cb..259fec62fa034 100644 --- a/src/stream/src/from_proto/stream_scan.rs +++ b/src/stream/src/from_proto/stream_scan.rs @@ -244,7 +244,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { ) .boxed() } - StreamScanType::ChainUnspecified => unreachable!(), + StreamScanType::Unspecified => unreachable!(), }; let rate_limit = node.get_rate_limit().cloned().ok(); Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) From aada1206083b1442759a6df9e6785b79dace2d0c Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Thu, 9 Nov 2023 16:47:24 +0800 Subject: [PATCH 23/77] fix: return notice when create source/sink if not exist (#13310) --- Cargo.lock | 1 + src/frontend/Cargo.toml | 1 + src/frontend/src/handler/create_index.rs | 22 +++++++++------------ src/frontend/src/handler/create_mv.rs | 21 ++++++++++---------- src/frontend/src/handler/create_sink.rs | 9 ++++++++- src/frontend/src/handler/create_source.rs | 9 ++++++++- src/frontend/src/handler/create_table.rs | 21 ++++++++++---------- src/frontend/src/handler/create_table_as.rs | 21 +++++++++----------- src/frontend/src/handler/create_view.rs | 19 ++++++++---------- src/frontend/src/session.rs | 21 ++++++++++++++------ 10 files changed, 79 insertions(+), 66 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2726498ea9856..7ccd83d3422fb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7732,6 +7732,7 @@ dependencies = [ "dyn-clone", "easy-ext", "educe", + "either", "enum-as-inner", "fixedbitset", "futures", diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index bae582dd06e24..73fcb205fafcb 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -29,6 +29,7 @@ downcast-rs = "1.2" dyn-clone = "1.0.14" easy-ext = "1" educe = "0.4" +either = "1" enum-as-inner = "0.6" fixedbitset = "0.4.2" futures = { version = "0.3", default-features = false, features = ["alloc"] } diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index 38975afd6fe52..3eb93f2900c95 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -15,6 +15,7 @@ use std::collections::{HashMap, HashSet}; use std::rc::Rc; +use either::Either; use fixedbitset::FixedBitSet; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; @@ -31,7 +32,6 @@ use risingwave_sqlparser::ast::{Ident, ObjectName, OrderByExpr}; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::root_catalog::SchemaPath; -use crate::catalog::CatalogError; use crate::expr::{Expr, ExprImpl, InputRef}; use crate::handler::privilege::ObjectCheckItem; use crate::handler::HandlerArgs; @@ -40,7 +40,7 @@ use crate::optimizer::plan_node::{Explain, LogicalProject, LogicalScan, StreamMa use crate::optimizer::property::{Cardinality, Distribution, Order, RequiredDist}; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot}; use crate::scheduler::streaming_manager::CreatingStreamingJobInfo; -use crate::session::{CheckRelationError, SessionImpl}; +use crate::session::SessionImpl; use crate::stream_fragmenter::build_graph; pub(crate) fn gen_create_index_plan( @@ -411,17 +411,13 @@ pub async fn handle_create_index( let (graph, index_table, index) = { { - match session.check_relation_name_duplicated(index_name.clone()) { - Err(CheckRelationError::Catalog(CatalogError::Duplicated(_, name))) - if if_not_exists => - { - return Ok(PgResponse::builder(StatementType::CREATE_INDEX) - .notice(format!("relation \"{}\" already exists, skipping", name)) - .into()); - } - Err(e) => return Err(e.into()), - Ok(_) => {} - }; + if let Either::Right(resp) = session.check_relation_name_duplicated( + index_name.clone(), + StatementType::CREATE_INDEX, + if_not_exists, + )? { + return Ok(resp); + } } let context = OptimizerContext::from_handler_args(handler_args); diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 75474ca576dd8..aeadf35aa2d8f 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use either::Either; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::acl::AclMode; @@ -23,7 +24,7 @@ use risingwave_sqlparser::ast::{EmitMode, Ident, ObjectName, Query}; use super::privilege::resolve_relation_privileges; use super::RwPgResponse; use crate::binder::{Binder, BoundQuery, BoundSetExpr}; -use crate::catalog::{check_valid_column_name, CatalogError}; +use crate::catalog::check_valid_column_name; use crate::handler::privilege::resolve_query_privileges; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::generic::GenericPlanRef; @@ -31,7 +32,7 @@ use crate::optimizer::plan_node::Explain; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, RelationCollectorVisitor}; use crate::planner::Planner; use crate::scheduler::streaming_manager::CreatingStreamingJobInfo; -use crate::session::{CheckRelationError, SessionImpl}; +use crate::session::SessionImpl; use crate::stream_fragmenter::build_graph; pub(super) fn get_column_names( @@ -154,15 +155,13 @@ pub async fn handle_create_mv( ) -> Result { let session = handler_args.session.clone(); - match session.check_relation_name_duplicated(name.clone()) { - Err(CheckRelationError::Catalog(CatalogError::Duplicated(_, name))) if if_not_exists => { - return Ok(PgResponse::builder(StatementType::CREATE_MATERIALIZED_VIEW) - .notice(format!("relation \"{}\" already exists, skipping", name)) - .into()); - } - Err(e) => return Err(e.into()), - Ok(_) => {} - }; + if let Either::Right(resp) = session.check_relation_name_duplicated( + name.clone(), + StatementType::CREATE_MATERIALIZED_VIEW, + if_not_exists, + )? { + return Ok(resp); + } let (mut table, graph) = { let context = OptimizerContext::from_handler_args(handler_args); diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index a4db476b6274a..5d520e4eb79c5 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -16,6 +16,7 @@ use std::collections::HashMap; use std::rc::Rc; use std::sync::LazyLock; +use either::Either; use itertools::Itertools; use maplit::{convert_args, hashmap}; use pgwire::pg_response::{PgResponse, StatementType}; @@ -188,7 +189,13 @@ pub async fn handle_create_sink( ) -> Result { let session = handle_args.session.clone(); - session.check_relation_name_duplicated(stmt.sink_name.clone())?; + if let Either::Right(resp) = session.check_relation_name_duplicated( + stmt.sink_name.clone(), + StatementType::CREATE_SINK, + stmt.if_not_exists, + )? { + return Ok(resp); + } let (sink, graph) = { let context = Rc::new(OptimizerContext::from_handler_args(handle_args)); diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index f7aa512b8bef7..d4f7633c55e21 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -16,6 +16,7 @@ use std::collections::{BTreeMap, HashMap}; use std::rc::Rc; use std::sync::LazyLock; +use either::Either; use itertools::Itertools; use maplit::{convert_args, hashmap}; use pgwire::pg_response::{PgResponse, StatementType}; @@ -1086,7 +1087,13 @@ pub async fn handle_create_source( ) -> Result { let session = handler_args.session.clone(); - session.check_relation_name_duplicated(stmt.source_name.clone())?; + if let Either::Right(resp) = session.check_relation_name_duplicated( + stmt.source_name.clone(), + StatementType::CREATE_SOURCE, + stmt.if_not_exists, + )? { + return Ok(resp); + } let db_name = session.database(); let (schema_name, name) = Binder::resolve_schema_qualified_name(db_name, stmt.source_name)?; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index e3d9490eecff2..9168f12bc992a 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -16,6 +16,7 @@ use std::collections::{BTreeMap, HashMap}; use std::rc::Rc; use anyhow::anyhow; +use either::Either; use fixedbitset::FixedBitSet; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; @@ -47,7 +48,7 @@ use crate::binder::{bind_data_type, bind_struct_field, Clause}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::table_catalog::TableVersion; -use crate::catalog::{check_valid_column_name, CatalogError, ColumnId}; +use crate::catalog::{check_valid_column_name, ColumnId}; use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; use crate::handler::create_source::{ bind_all_columns, bind_columns_from_source, bind_source_pk, bind_source_watermark, @@ -57,7 +58,7 @@ use crate::handler::HandlerArgs; use crate::optimizer::plan_node::{LogicalScan, LogicalSource}; use crate::optimizer::property::{Order, RequiredDist}; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot}; -use crate::session::{CheckRelationError, SessionImpl}; +use crate::session::SessionImpl; use crate::stream_fragmenter::build_graph; use crate::utils::resolve_privatelink_in_with_option; use crate::{Binder, TableCatalog, WithOptions}; @@ -954,15 +955,13 @@ pub async fn handle_create_table( session.notice_to_user("APPEND ONLY TABLE is currently an experimental feature."); } - match session.check_relation_name_duplicated(table_name.clone()) { - Err(CheckRelationError::Catalog(CatalogError::Duplicated(_, name))) if if_not_exists => { - return Ok(PgResponse::builder(StatementType::CREATE_TABLE) - .notice(format!("relation \"{}\" already exists, skipping", name)) - .into()); - } - Err(e) => return Err(e.into()), - Ok(_) => {} - }; + if let Either::Right(resp) = session.check_relation_name_duplicated( + table_name.clone(), + StatementType::CREATE_TABLE, + if_not_exists, + )? { + return Ok(resp); + } let (graph, source, table, job_type) = { let context = OptimizerContext::from_handler_args(handler_args); diff --git a/src/frontend/src/handler/create_table_as.rs b/src/frontend/src/handler/create_table_as.rs index accea93ca2837..9f1fe54a4d151 100644 --- a/src/frontend/src/handler/create_table_as.rs +++ b/src/frontend/src/handler/create_table_as.rs @@ -12,7 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use pgwire::pg_response::{PgResponse, StatementType}; +use either::Either; +use pgwire::pg_response::StatementType; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::ddl_service::TableJobType; @@ -21,10 +22,8 @@ use risingwave_sqlparser::ast::{ColumnDef, ObjectName, Query, Statement}; use super::{HandlerArgs, RwPgResponse}; use crate::binder::BoundStatement; -use crate::catalog::CatalogError; use crate::handler::create_table::{gen_create_table_plan_without_bind, ColumnIdGenerator}; use crate::handler::query::handle_query; -use crate::session::CheckRelationError; use crate::{build_graph, Binder, OptimizerContext}; pub async fn handle_create_as( @@ -43,15 +42,13 @@ pub async fn handle_create_as( } let session = handler_args.session.clone(); - match session.check_relation_name_duplicated(table_name.clone()) { - Err(CheckRelationError::Catalog(CatalogError::Duplicated(_, name))) if if_not_exists => { - return Ok(PgResponse::builder(StatementType::CREATE_TABLE) - .notice(format!("relation \"{}\" already exists, skipping", name)) - .into()); - } - Err(e) => return Err(e.into()), - Ok(_) => {} - }; + if let Either::Right(resp) = session.check_relation_name_duplicated( + table_name.clone(), + StatementType::CREATE_TABLE, + if_not_exists, + )? { + return Ok(resp); + } let mut col_id_gen = ColumnIdGenerator::new_initial(); diff --git a/src/frontend/src/handler/create_view.rs b/src/frontend/src/handler/create_view.rs index 004c9a116f91a..fe5785d0f98c7 100644 --- a/src/frontend/src/handler/create_view.rs +++ b/src/frontend/src/handler/create_view.rs @@ -14,6 +14,7 @@ //! Handle creation of logical (non-materialized) views. +use either::Either; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::error::Result; @@ -23,10 +24,8 @@ use risingwave_sqlparser::ast::{Ident, ObjectName, Query, Statement}; use super::RwPgResponse; use crate::binder::Binder; -use crate::catalog::CatalogError; use crate::handler::HandlerArgs; use crate::optimizer::OptimizerContext; -use crate::session::CheckRelationError; pub async fn handle_create_view( handler_args: HandlerArgs, @@ -43,15 +42,13 @@ pub async fn handle_create_view( let properties = handler_args.with_options.clone(); - match session.check_relation_name_duplicated(name.clone()) { - Err(CheckRelationError::Catalog(CatalogError::Duplicated(_, name))) if if_not_exists => { - return Ok(PgResponse::builder(StatementType::CREATE_VIEW) - .notice(format!("relation \"{}\" already exists, skipping", name)) - .into()); - } - Err(e) => return Err(e.into()), - Ok(_) => {} - }; + if let Either::Right(resp) = session.check_relation_name_duplicated( + name.clone(), + StatementType::CREATE_VIEW, + if_not_exists, + )? { + return Ok(resp); + } // plan the query to validate it and resolve dependencies let (dependent_relations, schema) = { diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 201882cc6416c..db3c5a49ba357 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -21,11 +21,12 @@ use std::sync::{Arc, Weak}; use std::time::{Duration, Instant}; use bytes::Bytes; +use either::Either; use parking_lot::{Mutex, RwLock, RwLockReadGuard}; use pgwire::net::{Address, AddressRef}; use pgwire::pg_field_descriptor::PgFieldDescriptor; use pgwire::pg_message::TransactionStatus; -use pgwire::pg_response::PgResponse; +use pgwire::pg_response::{PgResponse, StatementType}; use pgwire::pg_server::{ BoxedError, ExecContext, ExecContextGuard, Session, SessionId, SessionManager, UserAuthenticator, @@ -77,9 +78,9 @@ use crate::catalog::{ use crate::handler::extended_handle::{ handle_bind, handle_execute, handle_parse, Portal, PrepareStatement, }; -use crate::handler::handle; use crate::handler::privilege::ObjectCheckItem; use crate::handler::util::to_pg_field; +use crate::handler::{handle, RwPgResponse}; use crate::health_service::HealthServiceImpl; use crate::meta_client::{FrontendMetaClient, FrontendMetaClientImpl}; use crate::monitor::{FrontendMetrics, GLOBAL_FRONTEND_METRICS}; @@ -619,7 +620,9 @@ impl SessionImpl { pub fn check_relation_name_duplicated( &self, name: ObjectName, - ) -> std::result::Result<(), CheckRelationError> { + stmt_type: StatementType, + if_not_exists: bool, + ) -> std::result::Result, CheckRelationError> { let db_name = self.database(); let catalog_reader = self.env().catalog_reader().read_guard(); let (schema_name, relation_name) = { @@ -635,9 +638,15 @@ impl SessionImpl { }; (schema_name, relation_name) }; - catalog_reader.check_relation_name_duplicated(db_name, &schema_name, &relation_name)?; - - Ok(()) + match catalog_reader.check_relation_name_duplicated(db_name, &schema_name, &relation_name) { + Err(CatalogError::Duplicated(_, name)) if if_not_exists => Ok(Either::Right( + PgResponse::builder(stmt_type) + .notice(format!("relation \"{}\" already exists, skipping", name)) + .into(), + )), + Err(e) => Err(e.into()), + Ok(_) => Ok(Either::Left(())), + } } pub fn check_connection_name_duplicated(&self, name: ObjectName) -> Result<()> { From 7b9896cceb43a06687c5c6dfb6b2a8ba7a7f7a61 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 9 Nov 2023 17:37:24 +0800 Subject: [PATCH 24/77] feat(error): report the error's source chain through psql (#13264) Signed-off-by: Bugen Zhao --- Cargo.lock | 13 +++--- Cargo.toml | 2 +- Makefile.toml | 2 +- ci/Dockerfile | 2 +- ci/build-ci-image.sh | 2 +- ci/docker-compose.yml | 10 ++--- ci/scripts/run-e2e-test.sh | 8 ++++ e2e_test/batch/aggregate/sum.slt.part | 4 +- e2e_test/batch/basic/dml.slt.part | 2 +- e2e_test/batch/functions/pow.slt.part | 12 +++--- e2e_test/ddl/invalid_operation.slt | 3 +- e2e_test/ddl/table/generated_columns.slt.part | 8 ++-- e2e_test/error_ui/README.md | 12 ++++++ e2e_test/error_ui/main.slt | 41 +++++++++++++++++++ e2e_test/sink/append_only_sink.slt | 2 +- e2e_test/sink/kafka/avro.slt | 2 +- e2e_test/sink/kafka/protobuf.slt | 2 +- e2e_test/source/basic/datagen.slt | 4 +- e2e_test/source/basic/ddl.slt | 4 +- .../basic/old_row_format_syntax/datagen.slt | 4 +- .../basic/old_row_format_syntax/ddl.slt | 4 +- e2e_test/streaming/watermark.slt | 2 +- src/tests/simulation/Cargo.toml | 2 +- src/utils/pgwire/Cargo.toml | 1 + src/utils/pgwire/src/error.rs | 5 ++- src/utils/pgwire/src/pg_message.rs | 4 +- 26 files changed, 111 insertions(+), 46 deletions(-) create mode 100644 e2e_test/error_ui/README.md create mode 100644 e2e_test/error_ui/main.slt diff --git a/Cargo.lock b/Cargo.lock index 7ccd83d3422fb..1b4277c0e692c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5978,6 +5978,7 @@ dependencies = [ "risingwave_sqlparser", "tempfile", "thiserror", + "thiserror-ext", "tokio-openssl", "tokio-postgres", "tracing", @@ -9503,9 +9504,9 @@ dependencies = [ [[package]] name = "sqllogictest" -version = "0.17.2" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78ea1056caa9180e7e5727eed1a377d96c9f4615303fa82d2f4c202c64736dee" +checksum = "e67e8d8e7f43fc5d4c35be73549ea22ed1d712dcf39656a243144677297da5ec" dependencies = [ "async-trait", "educe", @@ -9999,9 +10000,9 @@ dependencies = [ [[package]] name = "thiserror-ext" -version = "0.0.1" +version = "0.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f28a4a7351f496662affc257826b85dd2a613406ce3cc2f07b849685e166d8c" +checksum = "764a8c5e60a634fe2173e20038dc4d11195559b4099c61fc60e489e813268a68" dependencies = [ "thiserror", "thiserror-ext-derive", @@ -10009,9 +10010,9 @@ dependencies = [ [[package]] name = "thiserror-ext-derive" -version = "0.0.1" +version = "0.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67621f6d39449754da63668ddd2423ad0c81c27434c16090f8805ad1db59b621" +checksum = "5741c7e65941d49975050cfc4d010e2786404a5e1cc130baa12132bfcf8efda4" dependencies = [ "proc-macro2", "quote", diff --git a/Cargo.toml b/Cargo.toml index 884b09edd7bda..ae1a3e2cdd70d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -121,7 +121,7 @@ arrow-buffer = "48" arrow-flight = "48" arrow-select = "48" arrow-ord = "48" -thiserror-ext = "0.0.1" +thiserror-ext = "0.0.5" tikv-jemalloc-ctl = { git = "https://github.com/risingwavelabs/jemallocator.git", rev = "64a2d9" } tikv-jemallocator = { git = "https://github.com/risingwavelabs/jemallocator.git", features = [ "profiling", diff --git a/Makefile.toml b/Makefile.toml index c60da632cd6aa..41b4036fac6e7 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -1333,7 +1333,7 @@ echo "All processes has exited." [tasks.slt] category = "RiseDev - SQLLogicTest" -install_crate = { version = "0.17.2", crate_name = "sqllogictest-bin", binary = "sqllogictest", test_arg = [ +install_crate = { version = "0.18.0", crate_name = "sqllogictest-bin", binary = "sqllogictest", test_arg = [ "--help", ], install_command = "binstall" } dependencies = ["check-risedev-env-file"] diff --git a/ci/Dockerfile b/ci/Dockerfile index dc75af612598e..427e6d68dc116 100644 --- a/ci/Dockerfile +++ b/ci/Dockerfile @@ -48,7 +48,7 @@ ENV CARGO_REGISTRIES_CRATES_IO_PROTOCOL=sparse RUN curl -L --proto '=https' --tlsv1.2 -sSf https://raw.githubusercontent.com/cargo-bins/cargo-binstall/main/install-from-binstall-release.sh | bash RUN cargo binstall -y --no-symlinks cargo-llvm-cov cargo-nextest cargo-hakari cargo-sort cargo-cache cargo-audit \ cargo-make@0.36.10 \ - sqllogictest-bin@0.17.2 \ + sqllogictest-bin@0.18.0 \ && cargo install sccache \ && cargo cache -a \ && rm -rf "/root/.cargo/registry/index" \ diff --git a/ci/build-ci-image.sh b/ci/build-ci-image.sh index 723fe00a76518..22f486aefdcfa 100755 --- a/ci/build-ci-image.sh +++ b/ci/build-ci-image.sh @@ -9,7 +9,7 @@ cd "$DIR" cat ../rust-toolchain # shellcheck disable=SC2155 -export BUILD_ENV_VERSION=v20231101 +export BUILD_ENV_VERSION=v20231109 export BUILD_TAG="public.ecr.aws/x5u3w5h6/rw-build-env:${BUILD_ENV_VERSION}" diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index 41ce645f567be..c6289789d5e67 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -71,7 +71,7 @@ services: retries: 5 source-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231101 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231109 depends_on: - mysql - db @@ -81,7 +81,7 @@ services: - ..:/risingwave sink-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231101 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231109 depends_on: - mysql - db @@ -93,12 +93,12 @@ services: - ..:/risingwave rw-build-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231101 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231109 volumes: - ..:/risingwave ci-flamegraph-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231101 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231109 # NOTE(kwannoel): This is used in order to permit # syscalls for `nperf` (perf_event_open), # so it can do CPU profiling. @@ -109,7 +109,7 @@ services: - ..:/risingwave regress-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231101 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231109 depends_on: db: condition: service_healthy diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index d6d26e0236ad0..b6b84d029aac0 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -105,6 +105,14 @@ sqllogictest -p 4566 -d dev './e2e_test/generated/**/*.slt' --junit "generated-$ echo "--- Kill cluster" cluster_stop +echo "--- e2e, $mode, error ui" +RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ +cluster_start +sqllogictest -p 4566 -d dev './e2e_test/error_ui/**/*.slt' + +echo "--- Kill cluster" +cluster_stop + echo "--- e2e, $mode, extended query" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ cluster_start diff --git a/e2e_test/batch/aggregate/sum.slt.part b/e2e_test/batch/aggregate/sum.slt.part index 825b684975986..1bb04023e04ab 100644 --- a/e2e_test/batch/aggregate/sum.slt.part +++ b/e2e_test/batch/aggregate/sum.slt.part @@ -77,8 +77,8 @@ select sum(d) from t; statement ok insert into t values (9000000000000000000000000000); -statement error Expr error: Numeric out of range +statement error Numeric out of range select sum(d) from t; statement ok -drop table t; \ No newline at end of file +drop table t; diff --git a/e2e_test/batch/basic/dml.slt.part b/e2e_test/batch/basic/dml.slt.part index 281eabf709c2f..a5a260169784c 100644 --- a/e2e_test/batch/basic/dml.slt.part +++ b/e2e_test/batch/basic/dml.slt.part @@ -121,7 +121,7 @@ select * from t order by v1; 1 1 aa 3 2 bb -statement error QueryError: Bind error: update modifying the PK column is unsupported +statement error update modifying the PK column is unsupported update t set (v3, v2) = (v3||v3, v1+v2); statement ok diff --git a/e2e_test/batch/functions/pow.slt.part b/e2e_test/batch/functions/pow.slt.part index 152cb65be0ea8..7ea01e2e1233f 100644 --- a/e2e_test/batch/functions/pow.slt.part +++ b/e2e_test/batch/functions/pow.slt.part @@ -46,11 +46,11 @@ select pow(100000, 0); statement error underflow select pow(100000, -200000000000000); -statement error QueryError: Expr error: Numeric out of range +statement error Numeric out of range select pow(100000, 200000000000000); -statement error QueryError: Expr error: Numeric out of range +statement error Numeric out of range select pow(-100000, 200000000000001); query R @@ -101,11 +101,11 @@ select 100000 ^ 0; statement error underflow select 100000 ^ -200000000000000; -statement error QueryError: Expr error: Numeric out of range +statement error Numeric out of range select 100000 ^ 200000000000000; -statement error QueryError: Expr error: Numeric out of range +statement error Numeric out of range select -100000 ^ 200000000000001; query RRRR @@ -231,10 +231,10 @@ select exp(2::smallint) ---- 7.38905609893065 -statement error QueryError: Expr error: Numeric out of range: overflow +statement error Numeric out of range: overflow select exp(10000000); -statement error QueryError: Expr error: Numeric out of range: underflow +statement error Numeric out of range: underflow select exp(-10000000); query TRR diff --git a/e2e_test/ddl/invalid_operation.slt b/e2e_test/ddl/invalid_operation.slt index f66aed1d3dc9a..df3f58eaf0354 100644 --- a/e2e_test/ddl/invalid_operation.slt +++ b/e2e_test/ddl/invalid_operation.slt @@ -253,7 +253,6 @@ SELECT * from msrc limit 0; # FIXME: improve the error message query error not found SELECT * from sink limit 0; ----- query I SELECT * from v limit 0; @@ -264,7 +263,7 @@ SELECT * from v limit 0; statement ok insert into t values (1); -statement error QueryError: Bind error: update modifying the PK column is unsupported +statement error update modifying the PK column is unsupported update t set v = 2; statement ok diff --git a/e2e_test/ddl/table/generated_columns.slt.part b/e2e_test/ddl/table/generated_columns.slt.part index feea107fe9ac1..0ba892b994865 100644 --- a/e2e_test/ddl/table/generated_columns.slt.part +++ b/e2e_test/ddl/table/generated_columns.slt.part @@ -14,7 +14,7 @@ select * from t1; 0 1 2 2.02 1 2 3 3.02 -statement error QueryError: Bind error: update modifying the generated column is unsupported +statement error update modifying the generated column is unsupported update t1 set v1 = 1; statement ok @@ -140,7 +140,7 @@ statement ok drop source t4; # create a table with generated column now -statement error QueryError: Bind error: failed to bind expression: now() +statement error failed to bind expression: now() CREATE TABLE t (v INT, t timestamptz as now()) WITH ( connector = 'datagen', fields.v.kind = 'sequence', @@ -151,9 +151,9 @@ CREATE TABLE t (v INT, t timestamptz as now()) WITH ( ) FORMAT PLAIN ENCODE JSON; # create a table with impure generated column as pk. -statement error QueryError: Bind error: Generated columns with impure expressions should not be part of the primary key. Here column "v2" is defined as part of the primary key. +statement error Generated columns with impure expressions should not be part of the primary key. Here column "v2" is defined as part of the primary key. CREATE TABLE t ( v1 INT, v2 timestamptz AS proctime(), PRIMARY KEY (v1, v2) -); \ No newline at end of file +); diff --git a/e2e_test/error_ui/README.md b/e2e_test/error_ui/README.md new file mode 100644 index 0000000000000..0aa51e5a9ee74 --- /dev/null +++ b/e2e_test/error_ui/README.md @@ -0,0 +1,12 @@ +# User Interface Test for Error Reporting + +The test cases in this directory act as snapshot tests for the error reporting, to ensure that the UI does not change unexpectedly. + +When you find the tests in this directory failing... + +- First, ensure that the changes to the error messages are expected and make them look better. +- Then, update the test cases by running: + ```bash + ./risedev slt './e2e_test/error_ui/**/*.slt' --override + ``` + Please note that the minimum required version of `sqllogictest` is 0.18 or higher. diff --git a/e2e_test/error_ui/main.slt b/e2e_test/error_ui/main.slt new file mode 100644 index 0000000000000..9a3df01ebd978 --- /dev/null +++ b/e2e_test/error_ui/main.slt @@ -0,0 +1,41 @@ +statement error +create function int_42() returns int as int_42 using link 'localhost:8815'; +---- +db error: ERROR: QueryError + +Caused by these errors (recent errors listed first): + 1: failed to connect to UDF service + 2: transport error + 3: error trying to connect + 4: invalid URL, scheme is missing + + +statement error +alter system set not_exist_key to value; +---- +db error: ERROR: QueryError + +Caused by this error: + 1: internal error: SystemParams error: unrecognized system param "not_exist_key" + + +query error +select v1 + v2 = v3; +---- +db error: ERROR: QueryError + +Caused by this error: + 1: Bind error: failed to bind expression: v1 + v2 = v3 + +Caused by: + Item not found: Invalid column: v1 + + +query error +select 1/0; +---- +db error: ERROR: QueryError + +Caused by these errors (recent errors listed first): + 1: Expr error + 2: Division by zero diff --git a/e2e_test/sink/append_only_sink.slt b/e2e_test/sink/append_only_sink.slt index 405ca132ae0a9..23bd1706465e3 100644 --- a/e2e_test/sink/append_only_sink.slt +++ b/e2e_test/sink/append_only_sink.slt @@ -22,7 +22,7 @@ create sink invalid_sink_type from t with (connector = 'blackhole', type = 'inva statement error `force_append_only` must be true or false create sink invalid_force_append_only from t with (connector = 'blackhole', force_append_only = 'invalid'); -statement error db error: ERROR: QueryError: Sink error: config error: unsupported sink type invalid +statement error unsupported sink type invalid create sink invalid_connector from t with (connector = 'invalid'); statement ok diff --git a/e2e_test/sink/kafka/avro.slt b/e2e_test/sink/kafka/avro.slt index a30d8b70fd4ba..88909135b9e53 100644 --- a/e2e_test/sink/kafka/avro.slt +++ b/e2e_test/sink/kafka/avro.slt @@ -71,7 +71,7 @@ create sink sink_err from into_kafka with ( format upsert encode avro ( schema.registry = 'http://message_queue:8081'); -statement error encode extra_column error: field not in avro +statement error field not in avro create sink sink_err as select 1 as extra_column, * from into_kafka with ( connector = 'kafka', topic = 'test-rw-sink-upsert-avro', diff --git a/e2e_test/sink/kafka/protobuf.slt b/e2e_test/sink/kafka/protobuf.slt index c6ccb2ac24416..9510c6e7fbf8b 100644 --- a/e2e_test/sink/kafka/protobuf.slt +++ b/e2e_test/sink/kafka/protobuf.slt @@ -76,7 +76,7 @@ format plain encode protobuf ( schema.location = 'file:///risingwave/proto-recursiv', message = 'recursive.AllTypes'); -statement error encode extra_column error: field not in proto +statement error field not in proto create sink sink_err as select 1 as extra_column with ( connector = 'kafka', topic = 'test-rw-sink-append-only-protobuf', diff --git a/e2e_test/source/basic/datagen.slt b/e2e_test/source/basic/datagen.slt index 14cbffb6b589b..dcda3718e49d5 100644 --- a/e2e_test/source/basic/datagen.slt +++ b/e2e_test/source/basic/datagen.slt @@ -182,9 +182,9 @@ statement ok drop table s1; # Do NOT allow With clause to contain a comma only. -statement error QueryError: sql parser error: Expected identifier.* +statement error Expected identifier.* create table s1 (v1 int) with (,) FORMAT PLAIN ENCODE JSON; # Do NOT allow an empty With clause. -statement error QueryError: sql parser error: Expected identifier.* +statement error Expected identifier.* create table s1 (v1 int) with () FORMAT PLAIN ENCODE JSON; diff --git a/e2e_test/source/basic/ddl.slt b/e2e_test/source/basic/ddl.slt index c6c1c0590d558..1bac32d5d512a 100644 --- a/e2e_test/source/basic/ddl.slt +++ b/e2e_test/source/basic/ddl.slt @@ -18,7 +18,7 @@ create source invalid_startup_mode ( ) FORMAT PLAIN ENCODE JSON; # TODO: Better to refine the error message. -statement error internal error: invalid digit found in string +statement error invalid digit found in string create source invalid_startup_timestamp ( column1 varchar ) with ( @@ -28,7 +28,7 @@ create source invalid_startup_timestamp ( properties.bootstrap.server = 'message_queue:29092' ) FORMAT PLAIN ENCODE JSON; -statement error db error: ERROR: QueryError: Protocol error: Schema definition is required, either from SQL or schema registry. +statement error Schema definition is required, either from SQL or schema registry. create source invalid_schema_definition with ( connector = 'kafka', diff --git a/e2e_test/source/basic/old_row_format_syntax/datagen.slt b/e2e_test/source/basic/old_row_format_syntax/datagen.slt index 629bc63728e74..267ae8eff4c66 100644 --- a/e2e_test/source/basic/old_row_format_syntax/datagen.slt +++ b/e2e_test/source/basic/old_row_format_syntax/datagen.slt @@ -182,9 +182,9 @@ statement ok drop table s1; # Do NOT allow With clause to contain a comma only. -statement error QueryError: sql parser error: Expected identifier.* +statement error Expected identifier.* create table s1 (v1 int) with (,) ROW FORMAT JSON; # Do NOT allow an empty With clause. -statement error QueryError: sql parser error: Expected identifier.* +statement error Expected identifier.* create table s1 (v1 int) with () ROW FORMAT JSON; diff --git a/e2e_test/source/basic/old_row_format_syntax/ddl.slt b/e2e_test/source/basic/old_row_format_syntax/ddl.slt index 6d1290463beeb..2ec7239e61d2c 100644 --- a/e2e_test/source/basic/old_row_format_syntax/ddl.slt +++ b/e2e_test/source/basic/old_row_format_syntax/ddl.slt @@ -18,7 +18,7 @@ create source invalid_startup_mode ( ) ROW FORMAT JSON; # TODO: Better to refine the error message. -statement error internal error: invalid digit found in string +statement error invalid digit found in string create source invalid_startup_timestamp ( column1 varchar ) with ( @@ -28,7 +28,7 @@ create source invalid_startup_timestamp ( properties.bootstrap.server = 'message_queue:29092' ) ROW FORMAT JSON; -statement error db error: ERROR: QueryError: Protocol error: Schema definition is required, either from SQL or schema registry. +statement error Schema definition is required, either from SQL or schema registry. create source invalid_schema_definition with ( connector = 'kafka', diff --git a/e2e_test/streaming/watermark.slt b/e2e_test/streaming/watermark.slt index d1a29b88188cf..5d8f189dfd965 100644 --- a/e2e_test/streaming/watermark.slt +++ b/e2e_test/streaming/watermark.slt @@ -55,7 +55,7 @@ statement ok drop table t; # create a watermark with different return type from column -statement error QueryError: Bind error: The return value type of the watermark expression must be identical to the watermark column data type. Current data type of watermark return value: `timestamp with time zone`, column `integer` +statement error The return value type of the watermark expression must be identical to the watermark column data type. Current data type of watermark return value: `timestamp with time zone`, column `integer` CREATE TABLE t ( v INT, WATERMARK FOR v AS to_timestamp(v) - INTERVAL '5' SECOND diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index f5b9d48eb0d20..15ae2e667ff6c 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -46,7 +46,7 @@ risingwave_sqlsmith = { workspace = true } serde = "1.0.188" serde_derive = "1.0.188" serde_json = "1.0.107" -sqllogictest = "0.17.2" +sqllogictest = "0.18" tempfile = "3" tikv-jemallocator = { workspace = true } tokio = { version = "0.2.24", package = "madsim-tokio" } diff --git a/src/utils/pgwire/Cargo.toml b/src/utils/pgwire/Cargo.toml index c7b6faf3c885d..c62def532c362 100644 --- a/src/utils/pgwire/Cargo.toml +++ b/src/utils/pgwire/Cargo.toml @@ -26,6 +26,7 @@ panic-message = "0.3" risingwave_common = { workspace = true } risingwave_sqlparser = { workspace = true } thiserror = "1" +thiserror-ext = { workspace = true } tokio = { version = "0.2", package = "madsim-tokio", features = ["rt", "macros"] } tokio-openssl = "0.6.3" tracing = "0.1" diff --git a/src/utils/pgwire/src/error.rs b/src/utils/pgwire/src/error.rs index ce3e3d05bef7d..c591d2f5a15ae 100644 --- a/src/utils/pgwire/src/error.rs +++ b/src/utils/pgwire/src/error.rs @@ -44,8 +44,9 @@ pub enum PsqlError { #[error(transparent)] Internal(BoxedError), - #[error("Panicked when processing: {0}.\n -This is a bug. We would appreciate a bug report at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Fbug&template=bug_report.yml.")] + #[error("Panicked when processing: {0} +This is a bug. We would appreciate a bug report at: + https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Fbug&template=bug_report.yml")] Panic(String), #[error("Unable to set up an ssl connection")] diff --git a/src/utils/pgwire/src/pg_message.rs b/src/utils/pgwire/src/pg_message.rs index b28d09116f94f..c38182db198f2 100644 --- a/src/utils/pgwire/src/pg_message.rs +++ b/src/utils/pgwire/src/pg_message.rs @@ -628,12 +628,14 @@ impl<'a> BeMessage<'a> { } BeMessage::ErrorResponse(error) => { + use thiserror_ext::AsReport; // For all the errors set Severity to Error and error code to // 'internal error'. // 'E' signalizes ErrorResponse messages buf.put_u8(b'E'); - let msg = error.to_string(); + // Format the error as a pretty report. + let msg = format!("{:#}", error.as_ref().as_report()); write_err_or_notice(buf, &ErrorOrNoticeMessage::internal_error(&msg))?; } From 117f14c20b2f742688d2691e5c4ab7f56d136425 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Thu, 9 Nov 2023 17:41:46 +0800 Subject: [PATCH 25/77] refactor(stream): use `ExecutorInfo` constructed by stream manager in `Project` and `ProjectSet` (#13342) Signed-off-by: Richard Chien --- src/stream/src/executor/integration_tests.rs | 9 +++- src/stream/src/executor/project.rs | 54 +++++++++---------- src/stream/src/executor/project_set.rs | 25 +++------ src/stream/src/from_proto/project.rs | 3 +- src/stream/src/from_proto/project_set.rs | 3 +- src/stream/src/task/stream_manager.rs | 34 ++++++------ .../tests/integration_tests/project_set.rs | 18 +++++-- 7 files changed, 75 insertions(+), 71 deletions(-) diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index 7346241fccab9..675cd37957fa0 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -155,13 +155,18 @@ async fn test_merger_sum_aggr() { let projection = ProjectExecutor::new( actor_ctx.clone(), + ExecutorInfo { + schema: Schema { + fields: vec![Field::unnamed(DataType::Int64)], + }, + pk_indices: vec![], + identity: format!("ProjectExecutor {:X}", 3), + }, aggregator, - vec![], vec![ // TODO: use the new streaming_if_null expression here, and add `None` tests NonStrictExpression::for_test(InputRefExpression::new(DataType::Int64, 1)), ], - 3, MultiMap::new(), vec![], 0.0, diff --git a/src/stream/src/executor/project.rs b/src/stream/src/executor/project.rs index 8cfebfecd3f33..3d27e9872a51e 100644 --- a/src/stream/src/executor/project.rs +++ b/src/stream/src/executor/project.rs @@ -14,10 +14,9 @@ use std::fmt::{Debug, Formatter}; -use itertools::Itertools; use multimap::MultiMap; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::Schema; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::ToOwnedDatum; use risingwave_common::util::iter_util::ZipEqFast; @@ -56,38 +55,19 @@ impl ProjectExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, + info: ExecutorInfo, input: Box, - pk_indices: PkIndices, exprs: Vec, - executor_id: u64, watermark_derivations: MultiMap, nondecreasing_expr_indices: Vec, materialize_selectivity_threshold: f64, ) -> Self { - let info = ExecutorInfo { - schema: input.schema().to_owned(), - pk_indices, - identity: "Project".to_owned(), - }; - - let schema = Schema { - fields: exprs - .iter() - .map(|e| Field::unnamed(e.return_type())) - .collect_vec(), - }; - let n_nondecreasing_exprs = nondecreasing_expr_indices.len(); - Self { input, inner: Inner { _ctx: ctx, - info: ExecutorInfo { - schema, - pk_indices: info.pk_indices, - identity: format!("ProjectExecutor {:X}", executor_id), - }, + info, exprs, watermark_derivations, nondecreasing_expr_indices, @@ -261,16 +241,23 @@ mod tests { ], }; let pk_indices = vec![0]; - let (mut tx, source) = MockSource::channel(schema, pk_indices.clone()); + let (mut tx, source) = MockSource::channel(schema, pk_indices); let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)"); + let info = ExecutorInfo { + schema: Schema { + fields: vec![Field::unnamed(DataType::Int64)], + }, + pk_indices: vec![], + identity: "ProjectExecutor".to_string(), + }; + let project = Box::new(ProjectExecutor::new( ActorContext::create(123), + info, Box::new(source), - pk_indices, vec![test_expr], - 1, MultiMap::new(), vec![], 0.0, @@ -348,12 +335,23 @@ mod tests { let b_expr = build_from_pretty("(subtract:int8 $0:int8 1:int8)"); let c_expr = NonStrictExpression::for_test(DummyNondecreasingExpr); + let info = ExecutorInfo { + schema: Schema { + fields: vec![ + Field::unnamed(DataType::Int64), + Field::unnamed(DataType::Int64), + Field::unnamed(DataType::Int64), + ], + }, + pk_indices: vec![], + identity: "ProjectExecutor".to_string(), + }; + let project = Box::new(ProjectExecutor::new( ActorContext::create(123), + info, Box::new(source), - vec![], vec![a_expr, b_expr, c_expr], - 1, MultiMap::from_iter(vec![(0, 0), (0, 1)].into_iter()), vec![2], 0.0, diff --git a/src/stream/src/executor/project_set.rs b/src/stream/src/executor/project_set.rs index e1000122af247..71b189fe00d3d 100644 --- a/src/stream/src/executor/project_set.rs +++ b/src/stream/src/executor/project_set.rs @@ -20,7 +20,7 @@ use futures_async_stream::try_stream; use multimap::MultiMap; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::bail; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::Schema; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::{DataType, Datum, DatumRef, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqFast; @@ -29,7 +29,7 @@ use risingwave_expr::table_function::ProjectSetSelectItem; use super::error::StreamExecutorError; use super::{ - ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndices, PkIndicesRef, + ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef, StreamExecutorResult, Watermark, }; use crate::common::StreamChunkBuilder; @@ -45,8 +45,9 @@ pub struct ProjectSetExecutor { } struct Inner { - info: ExecutorInfo, _ctx: ActorContextRef, + info: ExecutorInfo, + /// Expressions of the current project_section. select_list: Vec, chunk_size: usize, @@ -61,30 +62,16 @@ impl ProjectSetExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, + info: ExecutorInfo, input: Box, - pk_indices: PkIndices, select_list: Vec, - executor_id: u64, chunk_size: usize, watermark_derivations: MultiMap, nondecreasing_expr_indices: Vec, ) -> Self { - let mut fields = vec![Field::with_name(DataType::Int64, "projected_row_id")]; - fields.extend( - select_list - .iter() - .map(|expr| Field::unnamed(expr.return_type())), - ); - - let info = ExecutorInfo { - schema: Schema { fields }, - pk_indices, - identity: format!("ProjectSet {:X}", executor_id), - }; - let inner = Inner { - info, _ctx: ctx, + info, select_list, chunk_size, watermark_derivations, diff --git a/src/stream/src/from_proto/project.rs b/src/stream/src/from_proto/project.rs index e6656f357f6bd..12f09cac0977e 100644 --- a/src/stream/src/from_proto/project.rs +++ b/src/stream/src/from_proto/project.rs @@ -63,10 +63,9 @@ impl ExecutorBuilder for ProjectExecutorBuilder { let materialize_selectivity_threshold = if extremely_light { 0.0 } else { 0.5 }; Ok(ProjectExecutor::new( params.actor_context, + params.info, input, - params.pk_indices, project_exprs, - params.executor_id, watermark_derivations, nondecreasing_expr_indices, materialize_selectivity_threshold, diff --git a/src/stream/src/from_proto/project_set.rs b/src/stream/src/from_proto/project_set.rs index 33a7d0c3565d1..b2d0f0d148748 100644 --- a/src/stream/src/from_proto/project_set.rs +++ b/src/stream/src/from_proto/project_set.rs @@ -58,10 +58,9 @@ impl ExecutorBuilder for ProjectSetExecutorBuilder { let chunk_size = params.env.config().developer.chunk_size; Ok(ProjectSetExecutor::new( params.actor_context, + params.info, input, - params.pk_indices, select_list, - params.executor_id, chunk_size, watermark_derivations, nondecreasing_expr_indices, diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 7ee45a4d24f5e..a237e790588ed 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -123,9 +123,13 @@ impl risingwave_expr::expr::EvalErrorReport for ActorEvalErrorReport { pub struct ExecutorParams { pub env: StreamEnvironment, - /// Indices of primary keys - // TODO: directly use it for `ExecutorInfo` + /// Basic information about the executor. + pub info: ExecutorInfo, + + // TODO(rc): The following three fields will be removed. Executor builders should use `info` instead. + pub schema: Schema, pub pk_indices: PkIndices, + pub identity: String, /// Executor id, unique across all actors. pub executor_id: u64, @@ -137,14 +141,6 @@ pub struct ExecutorParams { // TODO: use it for `identity` pub op_info: String, - /// The output schema of the executor. - // TODO: directly use it for `ExecutorInfo` - pub schema: Schema, - - /// The identity of the executor, like `HashJoin 1234ABCD`. - // TODO: directly use it for `ExecutorInfo` - pub identity: String, - /// The input executor. pub input: Vec, @@ -167,11 +163,10 @@ pub struct ExecutorParams { impl Debug for ExecutorParams { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("ExecutorParams") - .field("pk_indices", &self.pk_indices) + .field("info", &self.info) .field("executor_id", &self.executor_id) .field("operator_id", &self.operator_id) .field("op_info", &self.op_info) - .field("schema", &self.schema) .field("input", &self.input.len()) .field("actor_id", &self.actor_context.id) .finish_non_exhaustive() @@ -557,7 +552,7 @@ impl LocalStreamManagerCore { // same. let executor_id = unique_executor_id(actor_context.id, node.operator_id); let operator_id = unique_operator_id(fragment_id, node.operator_id); - let schema = node.fields.iter().map(Field::from).collect(); + let schema: Schema = node.fields.iter().map(Field::from).collect(); let identity = format!("{} {:X}", node.get_node_body().unwrap(), executor_id); let eval_error_report = ActorEvalErrorReport { @@ -568,12 +563,21 @@ impl LocalStreamManagerCore { // Build the executor with params. let executor_params = ExecutorParams { env: env.clone(), + + info: ExecutorInfo { + schema: schema.clone(), + pk_indices: pk_indices.clone(), + identity: identity.clone(), + }, + + // TODO(rc): The following three fields will be removed. Executor builders should use `info` instead. + schema: schema.clone(), pk_indices: pk_indices.clone(), + identity: identity.clone(), + executor_id, operator_id, - identity: identity.clone(), op_info, - schema, input, fragment_id, executor_stats: self.streaming_metrics.clone(), diff --git a/src/stream/tests/integration_tests/project_set.rs b/src/stream/tests/integration_tests/project_set.rs index 61a879256108d..1b97b5f090b80 100644 --- a/src/stream/tests/integration_tests/project_set.rs +++ b/src/stream/tests/integration_tests/project_set.rs @@ -14,7 +14,7 @@ use multimap::MultiMap; use risingwave_expr::table_function::repeat; -use risingwave_stream::executor::ProjectSetExecutor; +use risingwave_stream::executor::{ExecutorInfo, ProjectSetExecutor}; use crate::prelude::*; @@ -34,17 +34,29 @@ fn create_executor() -> (MessageSender, BoxedMessageStream) { let tf1 = repeat(build_from_pretty("1:int4").into_inner(), 1); let tf2 = repeat(build_from_pretty("2:int4").into_inner(), 2); + let info = ExecutorInfo { + schema: Schema { + fields: vec![ + Field::unnamed(DataType::Int64), + Field::unnamed(DataType::Int64), + Field::unnamed(DataType::Int32), + Field::unnamed(DataType::Int32), + ], + }, + pk_indices: vec![], + identity: "ProjectSetExecutor".to_string(), + }; + let project_set = Box::new(ProjectSetExecutor::new( ActorContext::create(123), + info, Box::new(source), - vec![], vec![ test_expr.into(), test_expr_watermark.into(), tf1.into(), tf2.into(), ], - 1, CHUNK_SIZE, MultiMap::from_iter(std::iter::once((0, 1))), vec![], From 492076f2861db6ebcc79c4e3934b82eb5a553c6c Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Thu, 9 Nov 2023 18:54:37 +0800 Subject: [PATCH 26/77] fix(meta): collect meta node count as worker node (#13338) --- src/meta/src/manager/cluster.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index a31979c8871b0..0546231cb79fa 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -697,10 +697,11 @@ impl ClusterManagerCore { } fn count_worker_node(&self) -> HashMap { - const MONITORED_WORKER_TYPES: [WorkerType; 3] = [ + const MONITORED_WORKER_TYPES: [WorkerType; 4] = [ WorkerType::Compactor, WorkerType::ComputeNode, WorkerType::Frontend, + WorkerType::Meta, ]; let mut ret = HashMap::new(); self.workers From b8b812ddfaea4d8bb1bc8fe784890f8952c02a50 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Fri, 10 Nov 2023 10:02:34 +0800 Subject: [PATCH 27/77] chore: add memory limit & restarts in docker-compose.yml (#13343) --- docker/docker-compose-standalone.yml | 12 ++++++++++ docker/docker-compose-with-hdfs.yml | 32 +++++++++++++++++++++++++++ docker/docker-compose.yml | 33 ++++++++++++++++++++++++++++ 3 files changed, 77 insertions(+) diff --git a/docker/docker-compose-standalone.yml b/docker/docker-compose-standalone.yml index c70ec4c271310..b870589713d11 100644 --- a/docker/docker-compose-standalone.yml +++ b/docker/docker-compose-standalone.yml @@ -71,6 +71,13 @@ services: - bash -c 'printf \"GET / HTTP/1.1\n\n\" > /dev/tcp/127.0.0.1/5690; exit $$?;' interval: 1s timeout: 5s + restart: always + deploy: + resources: + limits: + memory: 28Gi + reservations: + memory: 28Gi etcd-0: image: "quay.io/coreos/etcd:v3.5.7" @@ -120,6 +127,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always grafana-0: image: "grafana/grafana-oss:latest" @@ -144,6 +152,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always minio-0: image: "quay.io/minio/minio:latest" @@ -189,6 +198,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always prometheus-0: image: "prom/prometheus:latest" @@ -216,6 +226,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always message_queue: image: "docker.vectorized.io/vectorized/redpanda:latest" @@ -255,6 +266,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always volumes: etcd-0: external: false diff --git a/docker/docker-compose-with-hdfs.yml b/docker/docker-compose-with-hdfs.yml index 4467b03a7c039..c44a650eb2ade 100644 --- a/docker/docker-compose-with-hdfs.yml +++ b/docker/docker-compose-with-hdfs.yml @@ -34,6 +34,13 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always + deploy: + resources: + limits: + memory: 2Gi + reservations: + memory: 1Gi compute-node-0: image: "ghcr.io/risingwavelabs/risingwave:RW_1.1_HADOOP2-x86_64" command: @@ -69,6 +76,13 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always + deploy: + resources: + limits: + memory: 26Gi + reservations: + memory: 26Gi etcd-0: image: "quay.io/coreos/etcd:v3.5.7" command: @@ -118,6 +132,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always frontend-node-0: image: "ghcr.io/risingwavelabs/risingwave:RW_1.1_HADOOP2-x86_64" command: @@ -150,6 +165,13 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always + deploy: + resources: + limits: + memory: 2Gi + reservations: + memory: 1Gi grafana-0: image: "grafana/grafana-oss:latest" command: [] @@ -173,6 +195,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always meta-node-0: image: "ghcr.io/risingwavelabs/risingwave:RW_1.1_HADOOP2-x86_64" command: @@ -219,6 +242,13 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always + deploy: + resources: + limits: + memory: 2Gi + reservations: + memory: 1Gi prometheus-0: image: "prom/prometheus:latest" command: @@ -245,6 +275,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always connector-node: image: ghcr.io/risingwavelabs/risingwave:RW_1.1_HADOOP2-x86_64 entrypoint: "/risingwave/bin/connector-node/start-service.sh" @@ -290,6 +321,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always volumes: etcd-0: external: false diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index fb2f23fb7739f..fe5c2fd9eda80 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -38,6 +38,13 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always + deploy: + resources: + limits: + memory: 2Gi + reservations: + memory: 1Gi compute-node-0: <<: *image command: @@ -75,6 +82,13 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always + deploy: + resources: + limits: + memory: 26Gi + reservations: + memory: 26Gi etcd-0: image: "quay.io/coreos/etcd:v3.5.7" command: @@ -123,6 +137,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always frontend-node-0: <<: *image command: @@ -157,6 +172,13 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always + deploy: + resources: + limits: + memory: 2Gi + reservations: + memory: 1Gi grafana-0: image: "grafana/grafana-oss:latest" command: [] @@ -180,6 +202,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always meta-node-0: <<: *image command: @@ -227,6 +250,13 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always + deploy: + resources: + limits: + memory: 2Gi + reservations: + memory: 1Gi minio-0: image: "quay.io/minio/minio:latest" command: @@ -271,6 +301,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always prometheus-0: image: "prom/prometheus:latest" command: @@ -297,6 +328,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always message_queue: image: "docker.vectorized.io/vectorized/redpanda:latest" command: @@ -335,6 +367,7 @@ services: interval: 1s timeout: 5s retries: 5 + restart: always volumes: etcd-0: external: false From 1abe9e82eb17d61aa9279a5cb8275af90083f504 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Fri, 10 Nov 2023 11:44:07 +0800 Subject: [PATCH 28/77] fix: remove connector-node from docker (#13344) --- docker/docker-compose-with-azblob.yml | 6 ------ docker/docker-compose-with-gcs.yml | 6 ------ docker/docker-compose-with-hdfs.yml | 11 ----------- docker/docker-compose-with-oss.yml | 6 ------ docker/docker-compose-with-s3.yml | 6 ------ docker/docker-compose.yml | 4 ---- docker/prometheus.yaml | 4 ---- 7 files changed, 43 deletions(-) diff --git a/docker/docker-compose-with-azblob.yml b/docker/docker-compose-with-azblob.yml index 91a88c6bbb9e7..52976672bd731 100644 --- a/docker/docker-compose-with-azblob.yml +++ b/docker/docker-compose-with-azblob.yml @@ -47,8 +47,6 @@ services: - etcd - "--etcd-endpoints" - "etcd-0:2388" - - "--connector-rpc-endpoint" - - "connector-node:50051" - "--state-store" - "hummock+azblob://" - "--data-directory" @@ -62,10 +60,6 @@ services: extends: file: docker-compose.yml service: prometheus- - connector-node: - extends: - file: docker-compose.yml - service: connector-node message_queue: extends: file: docker-compose.yml diff --git a/docker/docker-compose-with-gcs.yml b/docker/docker-compose-with-gcs.yml index a3c2b145ea3fa..86c5eac18bf61 100644 --- a/docker/docker-compose-with-gcs.yml +++ b/docker/docker-compose-with-gcs.yml @@ -47,8 +47,6 @@ services: - etcd - "--etcd-endpoints" - "etcd-0:2388" - - "--connector-rpc-endpoint" - - "connector-node:50051" - "--state-store" - "hummock+gcs://" - "--data-directory" @@ -62,10 +60,6 @@ services: extends: file: docker-compose.yml service: prometheus-0 - connector-node: - extends: - file: docker-compose.yml - service: connector-node message_queue: extends: file: docker-compose.yml diff --git a/docker/docker-compose-with-hdfs.yml b/docker/docker-compose-with-hdfs.yml index c44a650eb2ade..0929faf9fadd9 100644 --- a/docker/docker-compose-with-hdfs.yml +++ b/docker/docker-compose-with-hdfs.yml @@ -53,8 +53,6 @@ services: - "0.0.0.0:1222" - "--meta-address" - "http://meta-node-0:5690" - - "--connector-rpc-endpoint" - - "connector-node:50051" - "--config-path" - /risingwave.toml expose: @@ -212,8 +210,6 @@ services: - etcd - "--etcd-endpoints" - "etcd-0:2388" - - "--connector-rpc-endpoint" - - "connector-node:50051" - "--state-store" - "hummock+hdfs://" - "--data-directory" @@ -276,13 +272,6 @@ services: timeout: 5s retries: 5 restart: always - connector-node: - image: ghcr.io/risingwavelabs/risingwave:RW_1.1_HADOOP2-x86_64 - entrypoint: "/risingwave/bin/connector-node/start-service.sh" - ports: - - 50051 - - 50052 - container_name: connector-node message_queue: image: "docker.vectorized.io/vectorized/redpanda:latest" command: diff --git a/docker/docker-compose-with-oss.yml b/docker/docker-compose-with-oss.yml index 67729861815e7..1e4868964e147 100644 --- a/docker/docker-compose-with-oss.yml +++ b/docker/docker-compose-with-oss.yml @@ -47,8 +47,6 @@ services: - etcd - "--etcd-endpoints" - "etcd-0:2388" - - "--connector-rpc-endpoint" - - "connector-node:50051" - "--state-store" - "hummock+oss://" - "--data-directory" @@ -62,10 +60,6 @@ services: extends: file: docker-compose.yml service: prometheus-0 - connector-node: - extends: - file: docker-compose.yml - service: connector-node message_queue: extends: file: docker-compose.yml diff --git a/docker/docker-compose-with-s3.yml b/docker/docker-compose-with-s3.yml index 1e62d5e13d74a..4ba477c0d05a3 100644 --- a/docker/docker-compose-with-s3.yml +++ b/docker/docker-compose-with-s3.yml @@ -47,8 +47,6 @@ services: - etcd - "--etcd-endpoints" - "etcd-0:2388" - - "--connector-rpc-endpoint" - - "connector-node:50051" - "--state-store" - "hummock+s3://" - "--data-directory" @@ -62,10 +60,6 @@ services: extends: file: docker-compose.yml service: prometheus-0 - connector-node: - extends: - file: docker-compose.yml - service: connector-node message_queue: extends: file: docker-compose.yml diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index fe5c2fd9eda80..fd1532b0728ad 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -57,8 +57,6 @@ services: - "0.0.0.0:1222" - "--meta-address" - "http://meta-node-0:5690" - - "--connector-rpc-endpoint" - - "connector-node:50051" - "--config-path" - /risingwave.toml expose: @@ -219,8 +217,6 @@ services: - etcd - "--etcd-endpoints" - "etcd-0:2388" - - "--connector-rpc-endpoint" - - "connector-node:50051" - "--state-store" - "hummock+minio://hummockadmin:hummockadmin@minio-0:9301/hummock001" - "--data-directory" diff --git a/docker/prometheus.yaml b/docker/prometheus.yaml index f919847828839..d6607a321f3c7 100644 --- a/docker/prometheus.yaml +++ b/docker/prometheus.yaml @@ -36,7 +36,3 @@ scrape_configs: - job_name: redpanda static_configs: - targets: ["message_queue:9644"] - - - job_name: connector-node - static_configs: - - targets: ["connector-node:50052"] \ No newline at end of file From 80045051a7de3e0a7249b26a33ace96618f5c95b Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 10 Nov 2023 13:40:50 +0800 Subject: [PATCH 29/77] feat(error): preserve error's source chain across gRPC boundary (#13282) Signed-off-by: Bugen Zhao --- Cargo.lock | 28 ++++ Cargo.toml | 2 + e2e_test/error_ui/main.slt | 5 +- src/batch/src/error.rs | 3 +- src/common/Cargo.toml | 1 + src/common/src/error.rs | 66 ++++---- src/error/Cargo.toml | 22 +++ src/error/src/lib.rs | 21 +++ src/error/src/tonic.rs | 194 ++++++++++++++++++++++++ src/frontend/src/scheduler/error.rs | 2 +- src/meta/src/error.rs | 30 ++-- src/meta/src/hummock/error.rs | 3 +- src/meta/src/model/error.rs | 3 +- src/prost/Cargo.toml | 1 + src/prost/src/lib.rs | 7 +- src/rpc_client/Cargo.toml | 1 + src/rpc_client/src/error.rs | 16 +- src/storage/Cargo.toml | 1 + src/storage/src/filter_key_extractor.rs | 4 +- src/stream/src/error.rs | 4 +- src/stream/src/executor/error.rs | 2 +- 21 files changed, 353 insertions(+), 63 deletions(-) create mode 100644 src/error/Cargo.toml create mode 100644 src/error/src/lib.rs create mode 100644 src/error/src/tonic.rs diff --git a/Cargo.lock b/Cargo.lock index 1b4277c0e692c..b9e0b9a0b201a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7322,6 +7322,7 @@ dependencies = [ "regex", "reqwest", "risingwave_common_proc_macro", + "risingwave_error", "risingwave_pb", "rust_decimal", "rusty-fork", @@ -7633,6 +7634,21 @@ dependencies = [ "tracing-subscriber", ] +[[package]] +name = "risingwave_error" +version = "1.3.0-alpha" +dependencies = [ + "bincode 1.3.3", + "bytes", + "easy-ext", + "madsim-tonic", + "serde", + "serde-error", + "thiserror", + "thiserror-ext", + "tracing", +] + [[package]] name = "risingwave_expr" version = "1.3.0-alpha" @@ -8071,6 +8087,7 @@ dependencies = [ "prost-helpers", "serde", "strum 0.25.0", + "thiserror", "walkdir", "workspace-hack", ] @@ -8127,6 +8144,7 @@ dependencies = [ "moka", "rand", "risingwave_common", + "risingwave_error", "risingwave_hummock_sdk", "risingwave_pb", "static_assertions", @@ -8314,6 +8332,7 @@ dependencies = [ name = "risingwave_storage" version = "1.3.0-alpha" dependencies = [ + "anyhow", "arc-swap", "async-trait", "auto_enums", @@ -9018,6 +9037,15 @@ dependencies = [ "serde_derive", ] +[[package]] +name = "serde-error" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e988182713aeed6a619a88bca186f6d6407483485ffe44c869ee264f8eabd13f" +dependencies = [ + "serde", +] + [[package]] name = "serde-value" version = "0.7.0" diff --git a/Cargo.toml b/Cargo.toml index ae1a3e2cdd70d..4db4bbf02a703 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -11,6 +11,7 @@ members = [ "src/compute", "src/connector", "src/ctl", + "src/error", "src/expr/core", "src/expr/impl", "src/expr/macro", @@ -138,6 +139,7 @@ risingwave_compactor = { path = "./src/storage/compactor" } risingwave_compute = { path = "./src/compute" } risingwave_ctl = { path = "./src/ctl" } risingwave_connector = { path = "./src/connector" } +risingwave_error = { path = "./src/error" } risingwave_expr = { path = "./src/expr/core" } risingwave_expr_impl = { path = "./src/expr/impl" } risingwave_frontend = { path = "./src/frontend" } diff --git a/e2e_test/error_ui/main.slt b/e2e_test/error_ui/main.slt index 9a3df01ebd978..6dda0023a7827 100644 --- a/e2e_test/error_ui/main.slt +++ b/e2e_test/error_ui/main.slt @@ -15,8 +15,9 @@ alter system set not_exist_key to value; ---- db error: ERROR: QueryError -Caused by this error: - 1: internal error: SystemParams error: unrecognized system param "not_exist_key" +Caused by these errors (recent errors listed first): + 1: gRPC request to meta service failed: Internal error + 2: SystemParams error: unrecognized system param "not_exist_key" query error diff --git a/src/batch/src/error.rs b/src/batch/src/error.rs index 73d93706d299e..e2e30ce9bf5a8 100644 --- a/src/batch/src/error.rs +++ b/src/batch/src/error.rs @@ -17,6 +17,7 @@ use std::sync::Arc; pub use anyhow::anyhow; use risingwave_common::array::ArrayError; use risingwave_common::error::{ErrorCode, RwError}; +use risingwave_rpc_client::error::ToTonicStatus; use thiserror::Error; use tonic::Status; @@ -79,6 +80,6 @@ impl From for BatchError { impl<'a> From<&'a BatchError> for Status { fn from(err: &'a BatchError) -> Self { - Status::internal(err.to_string()) + err.to_status(tonic::Code::Internal, "batch") } } diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 1f8787c153a85..27b3a4249e3bc 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -73,6 +73,7 @@ rand = "0.8" regex = "1" reqwest = { version = "0.11", features = ["json"] } risingwave_common_proc_macro = { path = "./proc_macro" } +risingwave_error = { workspace = true } risingwave_pb = { workspace = true } rust_decimal = { version = "1", features = ["db-postgres", "maths"] } ryu = "1.0" diff --git a/src/common/src/error.rs b/src/common/src/error.rs index 70cfae626f6a4..160ee1a2c61a6 100644 --- a/src/common/src/error.rs +++ b/src/common/src/error.rs @@ -20,17 +20,14 @@ use std::io::Error as IoError; use std::time::{Duration, SystemTime}; use memcomparable::Error as MemComparableError; +use risingwave_error::tonic::{ToTonicStatus, TonicStatusWrapper}; use risingwave_pb::PbFieldNotFound; use thiserror::Error; use tokio::task::JoinError; -use tonic::Code; use crate::array::ArrayError; use crate::util::value_encoding::error::ValueEncodingError; -/// Header used to store serialized [`RwError`] in grpc status. -pub const RW_ERROR_GRPC_HEADER: &str = "risingwave-error-bin"; - const ERROR_SUPPRESSOR_RESET_DURATION: Duration = Duration::from_millis(60 * 60 * 1000); // 1h pub trait Error = std::error::Error + Send + Sync + 'static; @@ -126,10 +123,10 @@ pub enum ErrorCode { #[source] BoxedError, ), - #[error("RPC error: {0}")] + #[error(transparent)] RpcError( - #[source] - #[backtrace] + // #[backtrace] // TODO(error-handling): there's a limitation that `#[transparent]` can't be used with `#[backtrace]` if no `#[from]` + // `tonic::transport::Error`, `TonicStatusWrapper`, or `RpcError` BoxedError, ), #[error("Bind error: {0}")] @@ -195,12 +192,41 @@ pub struct RwError { impl From for tonic::Status { fn from(err: RwError) -> Self { - match &*err.inner { - ErrorCode::ExprError(e) => tonic::Status::invalid_argument(e.to_string()), - ErrorCode::PermissionDenied(e) => tonic::Status::permission_denied(e), - ErrorCode::InternalError(e) => tonic::Status::internal(e), - _ => tonic::Status::internal(err.to_string()), + use tonic::Code; + + let code = match &*err.inner { + ErrorCode::ExprError(_) => Code::InvalidArgument, + ErrorCode::PermissionDenied(_) => Code::PermissionDenied, + ErrorCode::InternalError(_) => Code::Internal, + _ => Code::Internal, + }; + + err.to_status_unnamed(code) + } +} + +impl From for RwError { + fn from(status: TonicStatusWrapper) -> Self { + use tonic::Code; + + let message = status.inner().message(); + + // TODO(error-handling): `message` loses the source chain. + match status.inner().code() { + Code::InvalidArgument => ErrorCode::InvalidParameterValue(message.to_string()), + Code::NotFound | Code::AlreadyExists => ErrorCode::CatalogError(status.into()), + Code::PermissionDenied => ErrorCode::PermissionDenied(message.to_string()), + Code::Cancelled => ErrorCode::SchedulerError(status.into()), + _ => ErrorCode::RpcError(status.into()), } + .into() + } +} + +impl From for RwError { + fn from(status: tonic::Status) -> Self { + // Always wrap the status. + Self::from(TonicStatusWrapper::new(status)) } } @@ -292,22 +318,6 @@ impl From for RwError { } } -impl From for RwError { - fn from(err: tonic::Status) -> Self { - match err.code() { - Code::InvalidArgument => { - ErrorCode::InvalidParameterValue(err.message().to_string()).into() - } - Code::NotFound | Code::AlreadyExists => { - ErrorCode::CatalogError(err.message().to_string().into()).into() - } - Code::PermissionDenied => ErrorCode::PermissionDenied(err.message().to_string()).into(), - Code::Cancelled => ErrorCode::SchedulerError(err.message().to_string().into()).into(), - _ => ErrorCode::InternalError(err.message().to_string()).into(), - } - } -} - impl From for RwError { fn from(err: tonic::transport::Error) -> Self { ErrorCode::RpcError(err.into()).into() diff --git a/src/error/Cargo.toml b/src/error/Cargo.toml new file mode 100644 index 0000000000000..13bb50a371853 --- /dev/null +++ b/src/error/Cargo.toml @@ -0,0 +1,22 @@ +[package] +name = "risingwave_error" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +keywords = { workspace = true } +license = { workspace = true } +repository = { workspace = true } + +[dependencies] +bincode = "1" +bytes = "1" +easy-ext = "1" +serde = "1" +serde-error = "0.1" +thiserror = "1" +thiserror-ext = { workspace = true } +tonic = { workspace = true } +tracing = "0.1" + +[lints] +workspace = true diff --git a/src/error/src/lib.rs b/src/error/src/lib.rs new file mode 100644 index 0000000000000..34bb1dde5dcc6 --- /dev/null +++ b/src/error/src/lib.rs @@ -0,0 +1,21 @@ +// 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. + +//! Error handling utilities. +//! +//! This will eventually replace the `RwError` in `risingwave_common`. + +#![feature(error_generic_member_access)] + +pub mod tonic; diff --git a/src/error/src/tonic.rs b/src/error/src/tonic.rs new file mode 100644 index 0000000000000..1b85a87643f52 --- /dev/null +++ b/src/error/src/tonic.rs @@ -0,0 +1,194 @@ +// 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 std::borrow::Cow; +use std::error::Error; +use std::sync::Arc; + +use serde::{Deserialize, Serialize}; +use tonic::metadata::{MetadataMap, MetadataValue}; + +/// The key of the metadata field that contains the serialized error. +const ERROR_KEY: &str = "risingwave-error-bin"; + +/// The service name that the error is from. Used to provide better error message. +type ServiceName = Cow<'static, str>; + +/// The error produced by the gRPC server and sent to the client on the wire. +#[derive(Debug, Serialize, Deserialize)] +struct ServerError { + error: serde_error::Error, + service_name: Option, +} + +impl std::fmt::Display for ServerError { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.error.fmt(f) + } +} + +impl std::error::Error for ServerError { + fn source(&self) -> Option<&(dyn Error + 'static)> { + self.error.source() + } +} + +fn to_status(error: &T, code: tonic::Code, service_name: Option) -> tonic::Status +where + T: ?Sized + std::error::Error, +{ + // Embed the whole error (`self`) and its source chain into the details field. + // At the same time, set the message field to the error message of `self` (without source chain). + // The redundancy of the current error's message is intentional in case the client ignores the `details` field. + let source = ServerError { + error: serde_error::Error::new(error), + service_name, + }; + let serialized = bincode::serialize(&source).unwrap(); + + let mut metadata = MetadataMap::new(); + metadata.insert_bin(ERROR_KEY, MetadataValue::from_bytes(&serialized)); + + let mut status = tonic::Status::with_metadata(code, error.to_string(), metadata); + // Set the source of `tonic::Status`, though it's not likely to be used. + // This is only available before serializing to the wire. That's why we need to manually embed it + // into the `details` field. + status.set_source(Arc::new(source)); + status +} + +// TODO(error-handling): disallow constructing `tonic::Status` directly with `new` by clippy. +#[easy_ext::ext(ToTonicStatus)] +impl T +where + T: ?Sized + std::error::Error, +{ + /// Convert the error to [`tonic::Status`] with the given [`tonic::Code`] and service name. + /// + /// The source chain is preserved by pairing with [`TonicStatusWrapper`]. + pub fn to_status( + &self, + code: tonic::Code, + service_name: impl Into, + ) -> tonic::Status { + to_status(self, code, Some(service_name.into())) + } + + /// Convert the error to [`tonic::Status`] with the given [`tonic::Code`] without specifying + /// the service name. Prefer [`to_status`] if possible. + /// + /// The source chain is preserved by pairing with [`TonicStatusWrapper`]. + pub fn to_status_unnamed(&self, code: tonic::Code) -> tonic::Status { + to_status(self, code, None) + } +} + +/// A wrapper of [`tonic::Status`] that provides better error message and extracts +/// the source chain from the `details` field. +#[derive(Debug)] +pub struct TonicStatusWrapper(tonic::Status); + +impl TonicStatusWrapper { + /// Create a new [`TonicStatusWrapper`] from the given [`tonic::Status`] and extract + /// the source chain from its `details` field. + pub fn new(mut status: tonic::Status) -> Self { + if status.source().is_none() { + if let Some(value) = status.metadata().get_bin(ERROR_KEY) { + if let Some(e) = value.to_bytes().ok().and_then(|serialized| { + bincode::deserialize::(serialized.as_ref()).ok() + }) { + status.set_source(Arc::new(e)); + } else { + tracing::warn!("failed to deserialize error from gRPC metadata"); + } + } + } + Self(status) + } + + /// Returns the reference to the inner [`tonic::Status`]. + pub fn inner(&self) -> &tonic::Status { + &self.0 + } + + /// Consumes `self` and returns the inner [`tonic::Status`]. + pub fn into_inner(self) -> tonic::Status { + self.0 + } +} + +impl From for TonicStatusWrapper { + fn from(status: tonic::Status) -> Self { + Self::new(status) + } +} + +impl std::fmt::Display for TonicStatusWrapper { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "gRPC request")?; + if let Some(service_name) = self + .source() + .and_then(|s| s.downcast_ref::()) + .and_then(|s| s.service_name.as_ref()) + { + write!(f, " to {} service", service_name)?; + } + write!(f, " failed: {}: {}", self.0.code(), self.0.message()) + } +} + +impl std::error::Error for TonicStatusWrapper { + fn source(&self) -> Option<&(dyn std::error::Error + 'static)> { + // Delegate to `self.0` as if we're transparent. + self.0.source() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_source_chain_preserved() { + #[derive(thiserror::Error, Debug)] + #[error("{message}")] + struct MyError { + message: &'static str, + source: Option>, + } + + let original = MyError { + message: "outer", + source: Some(Box::new(MyError { + message: "inner", + source: None, + })), + }; + + let server_status = original.to_status(tonic::Code::Internal, "test"); + let body = server_status.to_http(); + let client_status = tonic::Status::from_header_map(body.headers()).unwrap(); + + let wrapper = TonicStatusWrapper::new(client_status); + assert_eq!( + wrapper.to_string(), + "gRPC request to test service failed: Internal error: outer" + ); + + let source = wrapper.source().unwrap(); + assert!(source.is::()); + assert_eq!(source.to_string(), "outer"); + assert_eq!(source.source().unwrap().to_string(), "inner"); + } +} diff --git a/src/frontend/src/scheduler/error.rs b/src/frontend/src/scheduler/error.rs index 0d4cf534793a8..9bd6dbb676202 100644 --- a/src/frontend/src/scheduler/error.rs +++ b/src/frontend/src/scheduler/error.rs @@ -26,7 +26,7 @@ pub enum SchedulerError { #[error("Pin snapshot error: {0} fails to get epoch {1}")] PinSnapshot(QueryId, u64), - #[error("Rpc error: {0}")] + #[error(transparent)] RpcError( #[from] #[backtrace] diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index 0a92c5257e640..93c6c0cb97103 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -19,7 +19,7 @@ use aws_sdk_ec2::error::DisplayErrorContext; use risingwave_common::error::BoxedError; use risingwave_connector::sink::SinkError; use risingwave_pb::PbFieldNotFound; -use risingwave_rpc_client::error::RpcError; +use risingwave_rpc_client::error::{RpcError, ToTonicStatus}; use crate::hummock::error::Error as HummockError; use crate::manager::WorkerId; @@ -51,7 +51,7 @@ enum MetaErrorInner { HummockError, ), - #[error("Rpc error: {0}")] + #[error(transparent)] RpcError( #[from] #[backtrace] @@ -233,19 +233,19 @@ where impl From for tonic::Status { fn from(err: MetaError) -> Self { - match &*err.inner { - MetaErrorInner::PermissionDenied(_) => { - tonic::Status::permission_denied(err.to_string()) - } - MetaErrorInner::CatalogIdNotFound(_, _) => tonic::Status::not_found(err.to_string()), - MetaErrorInner::Duplicated(_, _) => tonic::Status::already_exists(err.to_string()), - MetaErrorInner::Unavailable(_) => tonic::Status::unavailable(err.to_string()), - MetaErrorInner::Cancelled(_) => tonic::Status::cancelled(err.to_string()), - MetaErrorInner::InvalidParameter(msg) => { - tonic::Status::invalid_argument(msg.to_owned()) - } - _ => tonic::Status::internal(err.to_string()), - } + use tonic::Code; + + let code = match &*err.inner { + MetaErrorInner::PermissionDenied(_) => Code::PermissionDenied, + MetaErrorInner::CatalogIdNotFound(_, _) => Code::NotFound, + MetaErrorInner::Duplicated(_, _) => Code::AlreadyExists, + MetaErrorInner::Unavailable(_) => Code::Unavailable, + MetaErrorInner::Cancelled(_) => Code::Cancelled, + MetaErrorInner::InvalidParameter(_) => Code::InvalidArgument, + _ => Code::Internal, + }; + + err.to_status(code, "meta") } } diff --git a/src/meta/src/hummock/error.rs b/src/meta/src/hummock/error.rs index 096180e7d272f..2de9faea1b131 100644 --- a/src/meta/src/hummock/error.rs +++ b/src/meta/src/hummock/error.rs @@ -14,6 +14,7 @@ use risingwave_hummock_sdk::{HummockContextId, HummockSstableObjectId}; use risingwave_object_store::object::ObjectError; +use risingwave_rpc_client::error::ToTonicStatus; use thiserror::Error; use crate::model::MetadataModelError; @@ -84,6 +85,6 @@ impl From for Error { impl From for tonic::Status { fn from(err: Error) -> Self { - tonic::Status::new(tonic::Code::Internal, format!("{}", err)) + err.to_status(tonic::Code::Internal, "hummock") } } diff --git a/src/meta/src/model/error.rs b/src/meta/src/model/error.rs index bb68726f4be95..3ad70c6c434f6 100644 --- a/src/meta/src/model/error.rs +++ b/src/meta/src/model/error.rs @@ -14,6 +14,7 @@ use anyhow::anyhow; use risingwave_pb::PbFieldNotFound; +use risingwave_rpc_client::error::ToTonicStatus; use thiserror::Error; use crate::storage::MetaStoreError; @@ -51,7 +52,7 @@ impl From for MetadataModelError { impl From for tonic::Status { fn from(e: MetadataModelError) -> Self { - tonic::Status::new(tonic::Code::Internal, format!("{}", e)) + e.to_status(tonic::Code::Internal, "meta") } } diff --git a/src/prost/Cargo.toml b/src/prost/Cargo.toml index d373207966640..8c4468d337942 100644 --- a/src/prost/Cargo.toml +++ b/src/prost/Cargo.toml @@ -14,6 +14,7 @@ prost = { workspace = true } prost-helpers = { path = "helpers" } serde = { version = "1", features = ["derive"] } strum = "0.25" +thiserror = "1" tonic = { workspace = true } [target.'cfg(not(madsim))'.dependencies] diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index a5e0cf0e82664..c29d452a1ae60 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -19,6 +19,8 @@ use std::str::FromStr; +use thiserror::Error; + #[rustfmt::skip] #[cfg_attr(madsim, path = "sim/catalog.rs")] pub mod catalog; @@ -149,12 +151,13 @@ pub mod backup_service_serde; #[path = "java_binding.serde.rs"] pub mod java_binding_serde; -#[derive(Clone, PartialEq, Eq, Debug)] +#[derive(Clone, PartialEq, Eq, Debug, Error)] +#[error("field `{0}` not found")] pub struct PbFieldNotFound(pub &'static str); impl From for tonic::Status { fn from(e: PbFieldNotFound) -> Self { - tonic::Status::new(tonic::Code::Internal, e.0) + tonic::Status::new(tonic::Code::Internal, e.to_string()) } } diff --git a/src/rpc_client/Cargo.toml b/src/rpc_client/Cargo.toml index b23d7a9405bbd..f49a9b26ad5ec 100644 --- a/src/rpc_client/Cargo.toml +++ b/src/rpc_client/Cargo.toml @@ -25,6 +25,7 @@ lru = "0.10.1" moka = { version = "0.12", features = ["future"] } rand = "0.8" risingwave_common = { workspace = true } +risingwave_error = { workspace = true } risingwave_hummock_sdk = { workspace = true } risingwave_pb = { workspace = true } static_assertions = "1" diff --git a/src/rpc_client/src/error.rs b/src/rpc_client/src/error.rs index 274edfb721020..8b10faab1f349 100644 --- a/src/rpc_client/src/error.rs +++ b/src/rpc_client/src/error.rs @@ -12,19 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub use anyhow::anyhow; use risingwave_common::error::{ErrorCode, RwError}; use thiserror::Error; pub type Result = std::result::Result; +// Re-export these types as they're commonly used together with `RpcError`. +pub use risingwave_error::tonic::{ToTonicStatus, TonicStatusWrapper}; + #[derive(Error, Debug)] pub enum RpcError { - #[error("Transport error: {0}")] - TransportError(#[source] Box), + #[error(transparent)] + TransportError(Box), - #[error("gRPC error ({}): {}", .0.code(), .0.message())] - GrpcStatus(#[source] Box), + #[error(transparent)] + GrpcStatus(Box), #[error(transparent)] Internal( @@ -44,14 +46,14 @@ impl From for RpcError { impl From for RpcError { fn from(s: tonic::Status) -> Self { - RpcError::GrpcStatus(Box::new(s)) + RpcError::GrpcStatus(Box::new(TonicStatusWrapper::new(s))) } } impl From for RwError { fn from(r: RpcError) -> Self { match r { - RpcError::GrpcStatus(status) => (*status).into(), + RpcError::GrpcStatus(status) => TonicStatusWrapper::into(*status), _ => ErrorCode::RpcError(r.into()).into(), } } diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index 9e6b8486fa30d..a3899824e9c87 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -14,6 +14,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] +anyhow = "1" arc-swap = "1" async-trait = "0.1" auto_enums = { version = "0.8", features = ["futures03"] } diff --git a/src/storage/src/filter_key_extractor.rs b/src/storage/src/filter_key_extractor.rs index 6538042566537..c98c06c287ce2 100644 --- a/src/storage/src/filter_key_extractor.rs +++ b/src/storage/src/filter_key_extractor.rs @@ -25,7 +25,7 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_hummock_sdk::info_in_release; use risingwave_hummock_sdk::key::{get_table_id, TABLE_PREFIX_LEN}; use risingwave_pb::catalog::Table; -use risingwave_rpc_client::error::{anyhow, Result as RpcResult, RpcError}; +use risingwave_rpc_client::error::{Result as RpcResult, RpcError}; use risingwave_rpc_client::MetaClient; use crate::hummock::{HummockError, HummockResult}; @@ -253,7 +253,7 @@ impl StateTableAccessor for RemoteTableAccessor { #[async_trait::async_trait] impl StateTableAccessor for FakeRemoteTableAccessor { async fn get_tables(&self, _table_ids: &[u32]) -> RpcResult> { - Err(RpcError::Internal(anyhow!( + Err(RpcError::Internal(anyhow::anyhow!( "fake accessor does not support fetch remote table" ))) } diff --git a/src/stream/src/error.rs b/src/stream/src/error.rs index 6b690fe56086d..ee0c969801c54 100644 --- a/src/stream/src/error.rs +++ b/src/stream/src/error.rs @@ -19,6 +19,7 @@ use risingwave_connector::error::ConnectorError; use risingwave_connector::sink::SinkError; use risingwave_expr::ExprError; use risingwave_pb::PbFieldNotFound; +use risingwave_rpc_client::error::ToTonicStatus; use risingwave_storage::error::StorageError; use crate::executor::StreamExecutorError; @@ -169,8 +170,7 @@ impl From for StreamError { impl From for tonic::Status { fn from(error: StreamError) -> Self { - // Only encode the error message without the backtrace. - tonic::Status::internal(error.inner.to_string()) + error.to_status(tonic::Code::Internal, "stream") } } diff --git a/src/stream/src/executor/error.rs b/src/stream/src/executor/error.rs index b38a994ea271c..fc2f02f0aede0 100644 --- a/src/stream/src/executor/error.rs +++ b/src/stream/src/executor/error.rs @@ -82,7 +82,7 @@ enum ErrorKind { SinkError, ), - #[error("RPC error: {0}")] + #[error(transparent)] RpcError( #[from] #[backtrace] From 4c78170f4473974ee41cdc8841cc2a4b5896ef2b Mon Sep 17 00:00:00 2001 From: Phoenix7Rise <81153420+keven-huang@users.noreply.github.com> Date: Fri, 10 Nov 2023 14:30:51 +0800 Subject: [PATCH 30/77] feat: set default message queue key to format plain (#13278) Co-authored-by: jiamin.huang --- e2e_test/source/basic/kafka.slt | 12 ++ src/connector/src/parser/avro/parser.rs | 140 ++++++++++-------- src/connector/src/parser/bytes_parser.rs | 19 ++- src/connector/src/parser/plain_parser.rs | 40 ++++- src/connector/src/parser/unified/util.rs | 18 ++- .../tests/testdata/output/batch_source.yaml | 4 +- .../tests/testdata/output/create_source.yaml | 16 +- .../testdata/output/emit_on_window_close.yaml | 20 +-- .../tests/testdata/output/explain.yaml | 8 +- .../testdata/output/generated_columns.yaml | 2 +- .../tests/testdata/output/struct_query.yaml | 2 +- .../tests/testdata/output/watermark.yaml | 18 +-- src/frontend/src/handler/create_source.rs | 22 ++- src/frontend/src/handler/show.rs | 1 + src/frontend/src/handler/util.rs | 14 +- 15 files changed, 216 insertions(+), 120 deletions(-) diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index 782b4cca7a7f9..ff1178cb9c156 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -363,6 +363,15 @@ create table s27 with ( scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE JSON (schema.location = 'file:///risingwave/json-complex-schema') +# currently _rw_key can be set as primary key +statement ok +create table s28 (id bytea, PRIMARY KEY(_rw_key)) with ( + connector = 'kafka', + topic = 'kafka_source_format_bytes', + properties.bootstrap.server = 'message_queue:29092', + scan.startup.mode = 'earliest' +) FORMAT PLAIN ENCODE BYTES + statement ok CREATE TABLE mongo_customers ( _id BIGINT PRIMARY KEY, @@ -830,6 +839,9 @@ drop source s24 statement ok drop table s27 +statement ok +drop table s28 + statement ok DROP TABLE mongo_customers; diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 61c55decfc040..273b4dcd57fcf 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -207,7 +207,7 @@ mod test { use apache_avro::{Codec, Days, Duration, Millis, Months, Reader, Schema, Writer}; use itertools::Itertools; use risingwave_common::array::Op; - use risingwave_common::catalog::ColumnId; + use risingwave_common::catalog::{ColumnId, DEFAULT_KEY_COLUMN_NAME}; use risingwave_common::row::Row; use risingwave_common::types::{DataType, Date, Interval, ScalarImpl, Timestamptz}; use risingwave_common::{error, try_match_expand}; @@ -220,10 +220,12 @@ mod test { AvroParserConfig, }; use crate::aws_auth::AwsAuthProps; + use crate::parser::bytes_parser::BytesAccessBuilder; use crate::parser::plain_parser::PlainParser; use crate::parser::unified::avro::unix_epoch_days; use crate::parser::{ - AccessBuilderImpl, EncodingType, SourceStreamChunkBuilder, SpecificParserConfig, + AccessBuilderImpl, BytesProperties, EncodingProperties, EncodingType, + SourceStreamChunkBuilder, SpecificParserConfig, }; use crate::source::SourceColumnDesc; @@ -309,6 +311,11 @@ mod test { let conf = new_avro_conf_from_local(file_name).await?; Ok(PlainParser { + key_builder: AccessBuilderImpl::Bytes(BytesAccessBuilder::new( + EncodingProperties::Bytes(BytesProperties { + column_name: Some(DEFAULT_KEY_COLUMN_NAME.into()), + }), + )?), payload_builder: AccessBuilderImpl::Avro(AvroAccessBuilder::new( conf, EncodingType::Value, @@ -332,68 +339,75 @@ mod test { let flush = writer.flush().unwrap(); assert!(flush > 0); let input_data = writer.into_inner().unwrap(); + // _rw_key test cases + let key_testcases = vec![Some(br#"r"#.to_vec()), Some(vec![]), None]; let columns = build_rw_columns(); - let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 1); - { - let writer = builder.row_writer(); - parser.parse_inner(input_data, writer).await.unwrap(); - } - let chunk = builder.finish(); - let (op, row) = chunk.rows().next().unwrap(); - assert_eq!(op, Op::Insert); - let row = row.into_owned_row(); - for (i, field) in record.fields.iter().enumerate() { - let value = field.clone().1; - match value { - Value::String(str) | Value::Union(_, box Value::String(str)) => { - assert_eq!(row[i], Some(ScalarImpl::Utf8(str.into_boxed_str()))); - } - Value::Boolean(bool_val) => { - assert_eq!(row[i], Some(ScalarImpl::Bool(bool_val))); - } - Value::Int(int_val) => { - assert_eq!(row[i], Some(ScalarImpl::Int32(int_val))); - } - Value::Long(i64_val) => { - assert_eq!(row[i], Some(ScalarImpl::Int64(i64_val))); - } - Value::Float(f32_val) => { - assert_eq!(row[i], Some(ScalarImpl::Float32(f32_val.into()))); - } - Value::Double(f64_val) => { - assert_eq!(row[i], Some(ScalarImpl::Float64(f64_val.into()))); - } - Value::Date(days) => { - assert_eq!( - row[i], - Some(ScalarImpl::Date( - Date::with_days(days + unix_epoch_days()).unwrap(), - )) - ); - } - Value::TimestampMillis(millis) => { - assert_eq!( - row[i], - Some(Timestamptz::from_millis(millis).unwrap().into()) - ); - } - Value::TimestampMicros(micros) => { - assert_eq!(row[i], Some(Timestamptz::from_micros(micros).into())); - } - Value::Bytes(bytes) => { - assert_eq!(row[i], Some(ScalarImpl::Bytea(bytes.into_boxed_slice()))); - } - Value::Duration(duration) => { - let months = u32::from(duration.months()) as i32; - let days = u32::from(duration.days()) as i32; - let usecs = (u32::from(duration.millis()) as i64) * 1000; // never overflows - assert_eq!( - row[i], - Some(Interval::from_month_day_usec(months, days, usecs).into()) - ); - } - _ => { - unreachable!() + for key_data in key_testcases { + let mut builder = SourceStreamChunkBuilder::with_capacity(columns.clone(), 1); + { + let writer = builder.row_writer(); + parser + .parse_inner(key_data, Some(input_data.clone()), writer) + .await + .unwrap(); + } + let chunk = builder.finish(); + let (op, row) = chunk.rows().next().unwrap(); + assert_eq!(op, Op::Insert); + let row = row.into_owned_row(); + for (i, field) in record.fields.iter().enumerate() { + let value = field.clone().1; + match value { + Value::String(str) | Value::Union(_, box Value::String(str)) => { + assert_eq!(row[i], Some(ScalarImpl::Utf8(str.into_boxed_str()))); + } + Value::Boolean(bool_val) => { + assert_eq!(row[i], Some(ScalarImpl::Bool(bool_val))); + } + Value::Int(int_val) => { + assert_eq!(row[i], Some(ScalarImpl::Int32(int_val))); + } + Value::Long(i64_val) => { + assert_eq!(row[i], Some(ScalarImpl::Int64(i64_val))); + } + Value::Float(f32_val) => { + assert_eq!(row[i], Some(ScalarImpl::Float32(f32_val.into()))); + } + Value::Double(f64_val) => { + assert_eq!(row[i], Some(ScalarImpl::Float64(f64_val.into()))); + } + Value::Date(days) => { + assert_eq!( + row[i], + Some(ScalarImpl::Date( + Date::with_days(days + unix_epoch_days()).unwrap(), + )) + ); + } + Value::TimestampMillis(millis) => { + assert_eq!( + row[i], + Some(Timestamptz::from_millis(millis).unwrap().into()) + ); + } + Value::TimestampMicros(micros) => { + assert_eq!(row[i], Some(Timestamptz::from_micros(micros).into())); + } + Value::Bytes(bytes) => { + assert_eq!(row[i], Some(ScalarImpl::Bytea(bytes.into_boxed_slice()))); + } + Value::Duration(duration) => { + let months = u32::from(duration.months()) as i32; + let days = u32::from(duration.days()) as i32; + let usecs = (u32::from(duration.millis()) as i64) * 1000; // never overflows + assert_eq!( + row[i], + Some(Interval::from_month_day_usec(months, days, usecs).into()) + ); + } + _ => { + unreachable!() + } } } } diff --git a/src/connector/src/parser/bytes_parser.rs b/src/connector/src/parser/bytes_parser.rs index 2a0b2f1b90f2a..3001c837882d1 100644 --- a/src/connector/src/parser/bytes_parser.rs +++ b/src/connector/src/parser/bytes_parser.rs @@ -55,11 +55,15 @@ mod tests { SourceStreamChunkBuilder, SpecificParserConfig, }; - fn get_payload() -> Vec> { - vec![br#"t"#.to_vec(), br#"random"#.to_vec()] + type Item = (Vec, Vec); + fn get_item() -> Vec { + vec![ + (br#"a"#.to_vec(), br#"t"#.to_vec()), + (br#"r"#.to_vec(), br#"random"#.to_vec()), + ] } - async fn test_bytes_parser(get_payload: fn() -> Vec>) { + async fn test_bytes_parser(get_item: fn() -> Vec) { let descs = vec![SourceColumnDesc::simple("id", DataType::Bytea, 0.into())]; let props = SpecificParserConfig { key_encoding_config: None, @@ -72,9 +76,12 @@ mod tests { let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 2); - for payload in get_payload() { + for item in get_item() { let writer = builder.row_writer(); - parser.parse_inner(payload, writer).await.unwrap(); + parser + .parse_inner(Some(item.0), Some(item.1), writer) + .await + .unwrap(); } let chunk = builder.finish(); @@ -100,6 +107,6 @@ mod tests { #[tokio::test] async fn test_bytes_parse_object_top_level() { - test_bytes_parser(get_payload).await; + test_bytes_parser(get_item).await; } } diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index defb7ef54a1e6..7448efb6f19d2 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -12,20 +12,22 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::catalog::DEFAULT_KEY_COLUMN_NAME; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{ErrorCode, Result, RwError}; -use super::unified::util::apply_row_accessor_on_stream_chunk_writer; +use super::bytes_parser::BytesAccessBuilder; +use super::unified::util::apply_key_val_accessor_on_stream_chunk_writer; use super::{ - AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, + AccessBuilderImpl, ByteStreamSourceParser, BytesProperties, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, }; -use crate::only_parse_payload; use crate::parser::ParserFormat; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; #[derive(Debug)] pub struct PlainParser { + pub key_builder: AccessBuilderImpl, pub payload_builder: AccessBuilderImpl, pub(crate) rw_columns: Vec, pub source_ctx: SourceContextRef, @@ -37,6 +39,11 @@ impl PlainParser { rw_columns: Vec, source_ctx: SourceContextRef, ) -> Result { + let key_builder = AccessBuilderImpl::Bytes(BytesAccessBuilder::new( + EncodingProperties::Bytes(BytesProperties { + column_name: Some(DEFAULT_KEY_COLUMN_NAME.into()), + }), + )?); let payload_builder = match props.encoding_config { EncodingProperties::Protobuf(_) | EncodingProperties::Avro(_) @@ -50,6 +57,7 @@ impl PlainParser { } }; Ok(Self { + key_builder, payload_builder, rw_columns, source_ctx, @@ -58,12 +66,28 @@ impl PlainParser { pub async fn parse_inner( &mut self, - payload: Vec, + key: Option>, + payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, ) -> Result<()> { - let accessor = self.payload_builder.generate_accessor(payload).await?; + // if key is empty, set it as vec![]su + let key_data = key.unwrap_or_default(); + // if payload is empty, report error + let payload_data = payload.ok_or_else(|| { + RwError::from(ErrorCode::InternalError( + "Empty payload with nonempty key".into(), + )) + })?; - apply_row_accessor_on_stream_chunk_writer(accessor, &mut writer).map_err(Into::into) + let key_accessor = self.key_builder.generate_accessor(key_data).await?; + let payload_accessor = self.payload_builder.generate_accessor(payload_data).await?; + apply_key_val_accessor_on_stream_chunk_writer( + DEFAULT_KEY_COLUMN_NAME, + key_accessor, + payload_accessor, + &mut writer, + ) + .map_err(Into::into) } } @@ -82,10 +106,10 @@ impl ByteStreamSourceParser for PlainParser { async fn parse_one<'a>( &'a mut self, - _key: Option>, + key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, ) -> Result<()> { - only_parse_payload!(self, payload, writer) + self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/unified/util.rs b/src/connector/src/parser/unified/util.rs index 92cf5da3ac81c..4117591aa9000 100644 --- a/src/connector/src/parser/unified/util.rs +++ b/src/connector/src/parser/unified/util.rs @@ -14,7 +14,7 @@ use risingwave_common::error::{ErrorCode, RwError}; -use super::{Access, AccessError, AccessResult, ChangeEvent}; +use super::{Access, AccessError, AccessImpl, AccessResult, ChangeEvent}; use crate::parser::unified::ChangeEventOperation; use crate::parser::SourceStreamChunkRowWriter; use crate::source::SourceColumnDesc; @@ -46,6 +46,22 @@ pub fn apply_row_accessor_on_stream_chunk_writer( writer.insert(|column| accessor.access(&[&column.name], Some(&column.data_type))) } +pub fn apply_key_val_accessor_on_stream_chunk_writer( + key_column_name: &str, + key_accessor: AccessImpl<'_, '_>, + val_accessor: AccessImpl<'_, '_>, + writer: &mut SourceStreamChunkRowWriter<'_>, +) -> AccessResult<()> { + let f = |column: &SourceColumnDesc| { + if column.name == key_column_name { + key_accessor.access(&[&column.name], Some(&column.data_type)) + } else { + val_accessor.access(&[&column.name], Some(&column.data_type)) + } + }; + writer.insert(f) +} + impl From for RwError { fn from(val: AccessError) -> Self { ErrorCode::InternalError(format!("AccessError: {:?}", val)).into() diff --git a/src/frontend/planner_test/tests/testdata/output/batch_source.yaml b/src/frontend/planner_test/tests/testdata/output/batch_source.yaml index e2c5e418a8624..7b4d42cba1350 100644 --- a/src/frontend/planner_test/tests/testdata/output/batch_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/batch_source.yaml @@ -3,11 +3,11 @@ select * from s logical_plan: |- LogicalProject { exprs: [id, value] } - └─LogicalSource { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: s, columns: [id, value, _rw_key, _rw_kafka_timestamp, _row_id], time_range: (Unbounded, Unbounded) } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [id, value] } - └─BatchSource { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchSource { source: s, columns: [id, value, _rw_key, _rw_kafka_timestamp, _row_id], filter: (None, None) } create_source: format: plain encode: protobuf diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index 50178ccb1c6e5..f440a21a906e9 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -20,11 +20,11 @@ scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE CSV (delimiter = ',', without_header = true); explain_output: | - StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } + StreamMaterialize { columns: [v1, v2, _rw_key(hidden), _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource { source: s0, columns: [v1, v2, _row_id] } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamDml { columns: [v1, v2, _rw_key, _row_id] } + └─StreamSource { source: s0, columns: [v1, v2, _rw_key, _row_id] } - id: csv_delimiter_tab sql: | explain create table s0 (v1 int, v2 varchar) with ( @@ -34,8 +34,8 @@ scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE CSV (delimiter = E'\t', without_header = true); explain_output: | - StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } + StreamMaterialize { columns: [v1, v2, _rw_key(hidden), _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource { source: s0, columns: [v1, v2, _row_id] } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamDml { columns: [v1, v2, _rw_key, _row_id] } + └─StreamSource { source: s0, columns: [v1, v2, _rw_key, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml b/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml index cd3019382bd66..cb7155cfa0a4f 100644 --- a/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml +++ b/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml @@ -19,17 +19,17 @@ └─StreamProject { exprs: [v1, min(v2), count(distinct v3)], output_watermarks: [v1] } └─StreamHashAgg [append_only] { group_key: [v1], aggs: [min(v2), count(distinct v3), count], output_watermarks: [v1] } └─StreamExchange { dist: HashShard(v1) } - └─StreamRowIdGen { row_id_index: 3 } + └─StreamRowIdGen { row_id_index: 4 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - 10:Int32) }], output_watermarks: [v1] } - └─StreamSource { source: t, columns: [v1, v2, v3, _row_id] } + └─StreamSource { source: t, columns: [v1, v2, v3, _rw_key, _row_id] } eowc_stream_plan: |- StreamMaterialize { columns: [v1, min, agg], stream_key: [v1], pk_columns: [v1], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamProject { exprs: [v1, min(v2), count(distinct v3)], output_watermarks: [v1] } └─StreamHashAgg [append_only, eowc] { group_key: [v1], aggs: [min(v2), count(distinct v3), count], output_watermarks: [v1] } └─StreamExchange { dist: HashShard(v1) } - └─StreamRowIdGen { row_id_index: 3 } + └─StreamRowIdGen { row_id_index: 4 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - 10:Int32) }], output_watermarks: [v1] } - └─StreamSource { source: t, columns: [v1, v2, v3, _row_id] } + └─StreamSource { source: t, columns: [v1, v2, v3, _rw_key, _row_id] } eowc_stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [v1, min, agg], stream_key: [v1], pk_columns: [v1], pk_conflict: NoCheck, watermark_columns: [v1] } @@ -42,10 +42,10 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamRowIdGen { row_id_index: 3 } + StreamRowIdGen { row_id_index: 4 } └── StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - 10:Int32) }], output_watermarks: [v1] } ├── state tables: [ 2 ] - └── StreamSource { source: t, columns: [v1, v2, v3, _row_id] } { source state table: 3 } + └── StreamSource { source: t, columns: [v1, v2, v3, _rw_key, _row_id] } { source state table: 3 } Table 0 ├── columns: [ v1, min(v2), count(distinct v3), count ] @@ -142,9 +142,9 @@ └─StreamEowcSort { sort_column: tm } └─StreamExchange { dist: HashShard(b) } └─StreamProject { exprs: [a, b, tm, _row_id], output_watermarks: [tm] } - └─StreamRowIdGen { row_id_index: 3 } + └─StreamRowIdGen { row_id_index: 4 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: tm, expr: (tm - '00:05:00':Interval) }], output_watermarks: [tm] } - └─StreamSource { source: t, columns: [a, b, tm, _row_id] } + └─StreamSource { source: t, columns: [a, b, tm, _rw_key, _row_id] } eowc_stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [lag, _row_id(hidden), b(hidden)], stream_key: [_row_id, b], pk_columns: [_row_id, b], pk_conflict: NoCheck } @@ -157,10 +157,10 @@ Fragment 1 StreamProject { exprs: [a, b, tm, _row_id], output_watermarks: [tm] } - └── StreamRowIdGen { row_id_index: 3 } + └── StreamRowIdGen { row_id_index: 4 } └── StreamWatermarkFilter { watermark_descs: [Desc { column: tm, expr: (tm - '00:05:00':Interval) }], output_watermarks: [tm] } ├── state tables: [ 2 ] - └── StreamSource { source: t, columns: [a, b, tm, _row_id] } { source state table: 3 } + └── StreamSource { source: t, columns: [a, b, tm, _rw_key, _row_id] } { source state table: 3 } Table 0 ├── columns: [ a, b, tm, _row_id ] diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index 63899bc26c0e4..d3ac80df9b574 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -198,8 +198,8 @@ - sql: | explain create table t (v1 int, v2 varchar) with ( connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest' ) FORMAT PLAIN ENCODE JSON; explain_output: | - StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } + StreamMaterialize { columns: [v1, v2, _rw_key(hidden), _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource { source: t, columns: [v1, v2, _row_id] } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamDml { columns: [v1, v2, _rw_key, _row_id] } + └─StreamSource { source: t, columns: [v1, v2, _rw_key, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml index 6e2976cc29712..2fbfb1d8539b4 100644 --- a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml +++ b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml @@ -16,7 +16,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [(v2 + 1:Int32) as $expr1] } - └─BatchSource { source: s1, columns: [v2, _row_id], filter: (None, None) } + └─BatchSource { source: s1, columns: [v2, _rw_key, _row_id], filter: (None, None) } - name: select proctime() sql: | select proctime(); diff --git a/src/frontend/planner_test/tests/testdata/output/struct_query.yaml b/src/frontend/planner_test/tests/testdata/output/struct_query.yaml index bb7f8b95cc9ff..2b9f2de1b423b 100644 --- a/src/frontend/planner_test/tests/testdata/output/struct_query.yaml +++ b/src/frontend/planner_test/tests/testdata/output/struct_query.yaml @@ -396,7 +396,7 @@ logical_plan: |- LogicalProject { exprs: [s.v1, s.v2, s.v3] } └─LogicalFilter { predicate: (s.v3 = Row(1:Int32, 2:Int32, Row(1:Int32, 2:Int32, 3:Int32))) } - └─LogicalScan { table: s, columns: [s.v1, s.v2, s.v3, s._row_id] } + └─LogicalScan { table: s, columns: [s.v1, s.v2, s.v3, s._rw_key, s._row_id] } create_table_with_connector: format: plain encode: protobuf diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml index d57d41fa76bc3..e56f4eff7496f 100644 --- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml @@ -5,22 +5,22 @@ select t.v1 - INTERVAL '2' SECOND as v1 from t; logical_plan: |- LogicalProject { exprs: [(v1 - '00:00:02':Interval) as $expr1] } - └─LogicalSource { source: t, columns: [v1, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: t, columns: [v1, _rw_key, _row_id], time_range: (Unbounded, Unbounded) } stream_plan: |- StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamProject { exprs: [SubtractWithTimeZone(v1, '00:00:02':Interval, 'UTC':Varchar) as $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamRowIdGen { row_id_index: 1 } + └─StreamRowIdGen { row_id_index: 2 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - '00:00:01':Interval) }], output_watermarks: [v1] } - └─StreamSource { source: t, columns: [v1, _row_id] } + └─StreamSource { source: t, columns: [v1, _rw_key, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } ├── materialized table: 4294967294 └── StreamProject { exprs: [SubtractWithTimeZone(v1, '00:00:02':Interval, 'UTC':Varchar) as $expr1, _row_id], output_watermarks: [$expr1] } - └── StreamRowIdGen { row_id_index: 1 } + └── StreamRowIdGen { row_id_index: 2 } └── StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - '00:00:01':Interval) }], output_watermarks: [v1] } ├── state tables: [ 0 ] - └── StreamSource { source: t, columns: [v1, _row_id] } { source state table: 1 } + └── StreamSource { source: t, columns: [v1, _rw_key, _row_id] } { source state table: 1 } Table 0 ├── columns: [ vnode, offset ] @@ -48,11 +48,11 @@ sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only with (connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest') FORMAT PLAIN ENCODE JSON; explain_output: | - StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } - └─StreamRowIdGen { row_id_index: 1 } + StreamMaterialize { columns: [v1, _rw_key(hidden), _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } + └─StreamRowIdGen { row_id_index: 2 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - '00:00:01':Interval) }], output_watermarks: [v1] } - └─StreamDml { columns: [v1, _row_id] } - └─StreamSource { source: t, columns: [v1, _row_id] } + └─StreamDml { columns: [v1, _rw_key, _row_id] } + └─StreamSource { source: t, columns: [v1, _rw_key, _row_id] } - name: watermark on append only table without source sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index d4f7633c55e21..9e5f80a9a6edd 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -62,7 +62,9 @@ use crate::handler::create_table::{ bind_pk_on_relation, bind_sql_column_constraints, bind_sql_columns, bind_sql_pk_names, ensure_table_constraints_supported, ColumnIdGenerator, }; -use crate::handler::util::{get_connector, is_cdc_connector, is_kafka_connector}; +use crate::handler::util::{ + get_connector, is_cdc_connector, is_kafka_connector, is_key_mq_connector, +}; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::{LogicalSource, ToStream, ToStreamContext}; use crate::session::SessionImpl; @@ -577,6 +579,7 @@ pub(crate) async fn bind_columns_from_source( .join(","), )))); } + Ok(res) } @@ -704,12 +707,18 @@ pub(crate) async fn bind_source_pk( let sql_defined_pk = !sql_defined_pk_names.is_empty(); let res = match (&source_schema.format, &source_schema.row_encode) { - (Format::Native, Encode::Native) | (Format::Plain, _) => sql_defined_pk_names, + (Format::Native, Encode::Native) => sql_defined_pk_names, + (Format::Plain, _) => { + if is_key_mq_connector(with_properties) { + add_default_key_column(columns); + } + sql_defined_pk_names + } (Format::Upsert, Encode::Json) => { if sql_defined_pk { sql_defined_pk_names } else { - add_upsert_default_key_column(columns); + add_default_key_column(columns); vec![DEFAULT_KEY_COLUMN_NAME.into()] } } @@ -727,7 +736,7 @@ pub(crate) async fn bind_source_pk( extracted_pk_names } else { // For upsert avro, if we can't extract pk from schema, use message key as primary key - add_upsert_default_key_column(columns); + add_default_key_column(columns); vec![DEFAULT_KEY_COLUMN_NAME.into()] } } @@ -822,7 +831,7 @@ fn check_and_add_timestamp_column( } } -fn add_upsert_default_key_column(columns: &mut Vec) { +fn add_default_key_column(columns: &mut Vec) { let column = ColumnCatalog { column_desc: ColumnDesc { data_type: DataType::Bytea, @@ -1256,7 +1265,7 @@ pub mod tests { use risingwave_common::catalog::{ cdc_table_name_column_name, offset_column_name, row_id_column_name, DEFAULT_DATABASE_NAME, - DEFAULT_SCHEMA_NAME, + DEFAULT_KEY_COLUMN_NAME, DEFAULT_SCHEMA_NAME, }; use risingwave_common::types::DataType; @@ -1298,6 +1307,7 @@ pub mod tests { let row_id_col_name = row_id_column_name(); let expected_columns = maplit::hashmap! { row_id_col_name.as_str() => DataType::Serial, + DEFAULT_KEY_COLUMN_NAME => DataType::Bytea, "id" => DataType::Int32, "zipcode" => DataType::Int64, "rate" => DataType::Float32, diff --git a/src/frontend/src/handler/show.rs b/src/frontend/src/handler/show.rs index 2957519e39dff..977d2cb08a8bd 100644 --- a/src/frontend/src/handler/show.rs +++ b/src/frontend/src/handler/show.rs @@ -467,6 +467,7 @@ mod tests { "country".into() => "test.Country".into(), "_rw_kafka_timestamp".into() => "timestamp with time zone".into(), "_row_id".into() => "serial".into(), + "_rw_key".into() => "bytea".into() }; assert_eq!(columns, expected_columns); diff --git a/src/frontend/src/handler/util.rs b/src/frontend/src/handler/util.rs index 4ddb5419357ff..80ed4b19655bc 100644 --- a/src/frontend/src/handler/util.rs +++ b/src/frontend/src/handler/util.rs @@ -31,7 +31,7 @@ use risingwave_common::error::{ErrorCode, Result as RwResult}; use risingwave_common::row::Row as _; use risingwave_common::types::{DataType, ScalarRefImpl, Timestamptz}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_connector::source::KAFKA_CONNECTOR; +use risingwave_connector::source::{KAFKA_CONNECTOR, KINESIS_CONNECTOR, PULSAR_CONNECTOR}; use risingwave_sqlparser::ast::display_comma_separated; use crate::catalog::IndexCatalog; @@ -254,6 +254,18 @@ pub fn is_kafka_connector(with_properties: &HashMap) -> bool { connector == KAFKA_CONNECTOR } +#[inline(always)] +pub fn is_key_mq_connector(with_properties: &HashMap) -> bool { + let Some(connector) = get_connector(with_properties) else { + return false; + }; + + matches!( + connector.as_str(), + KINESIS_CONNECTOR | PULSAR_CONNECTOR | KAFKA_CONNECTOR + ) +} + #[inline(always)] pub fn is_cdc_connector(with_properties: &HashMap) -> bool { let Some(connector) = get_connector(with_properties) else { From e71d94fd292b428da3b831237b4f0611314a5426 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 10 Nov 2023 15:22:14 +0800 Subject: [PATCH 31/77] refactor(error): show pretty error in planner test (#13348) Signed-off-by: Bugen Zhao --- Cargo.lock | 9 +- Cargo.toml | 2 +- .../batch/duckdb/cte/test_cte.test.slt.part | 2 +- e2e_test/ddl/table/generated_columns.slt.part | 2 +- e2e_test/error_ui/main.slt | 8 +- src/common/src/error.rs | 9 ++ src/frontend/planner_test/Cargo.toml | 1 + src/frontend/planner_test/src/lib.rs | 21 ++-- .../tests/testdata/output/agg.yaml | 100 +++++++++--------- .../tests/testdata/output/array.yaml | 52 ++++----- .../tests/testdata/output/basic_query.yaml | 14 ++- .../testdata/output/concat_op_dispatch.yaml | 8 +- .../tests/testdata/output/expr.yaml | 94 ++++++++-------- .../tests/testdata/output/format.yaml | 22 +++- .../testdata/output/generated_columns.yaml | 4 +- .../tests/testdata/output/join.yaml | 4 +- .../tests/testdata/output/nexmark.yaml | 4 +- .../tests/testdata/output/nexmark_source.yaml | 8 +- .../output/nexmark_temporal_filter.yaml | 8 +- .../testdata/output/nexmark_watermark.yaml | 8 +- .../tests/testdata/output/order_by.yaml | 8 +- .../testdata/output/over_window_function.yaml | 40 +++---- .../tests/testdata/output/project_set.yaml | 20 ++-- .../tests/testdata/output/range_scan.yaml | 30 +++++- .../tests/testdata/output/select_except.yaml | 8 +- .../testdata/output/struct_field_access.yaml | 16 ++- .../tests/testdata/output/struct_query.yaml | 16 +-- .../tests/testdata/output/subquery.yaml | 32 +++--- .../tests/testdata/output/subquery_expr.yaml | 13 +-- .../output/subquery_expr_correlated.yaml | 4 +- .../tests/testdata/output/sysinfo_funcs.yaml | 4 +- .../testdata/output/table_primary_key.yaml | 4 +- .../tests/testdata/output/types.yaml | 16 +-- src/frontend/src/binder/expr/mod.rs | 48 ++------- src/utils/pgwire/src/pg_message.rs | 2 +- 35 files changed, 330 insertions(+), 311 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b9e0b9a0b201a..8728de3a0e751 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8109,6 +8109,7 @@ dependencies = [ "serde_with", "serde_yaml", "tempfile", + "thiserror-ext", "walkdir", "workspace-hack", ] @@ -10028,9 +10029,9 @@ dependencies = [ [[package]] name = "thiserror-ext" -version = "0.0.5" +version = "0.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "764a8c5e60a634fe2173e20038dc4d11195559b4099c61fc60e489e813268a68" +checksum = "b253f2e76cc35b73c1e1b9fd4e28d920f47e27b33b8b3e455bacc62cc0e40bf5" dependencies = [ "thiserror", "thiserror-ext-derive", @@ -10038,9 +10039,9 @@ dependencies = [ [[package]] name = "thiserror-ext-derive" -version = "0.0.5" +version = "0.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5741c7e65941d49975050cfc4d010e2786404a5e1cc130baa12132bfcf8efda4" +checksum = "11737e1864ac6e6ec08409cdaba243912693de6b5c7fac56a6aeabb0d73a4ad3" dependencies = [ "proc-macro2", "quote", diff --git a/Cargo.toml b/Cargo.toml index 4db4bbf02a703..ee4d8b06774ed 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -122,7 +122,7 @@ arrow-buffer = "48" arrow-flight = "48" arrow-select = "48" arrow-ord = "48" -thiserror-ext = "0.0.5" +thiserror-ext = "0.0.6" tikv-jemalloc-ctl = { git = "https://github.com/risingwavelabs/jemallocator.git", rev = "64a2d9" } tikv-jemallocator = { git = "https://github.com/risingwavelabs/jemallocator.git", features = [ "profiling", diff --git a/e2e_test/batch/duckdb/cte/test_cte.test.slt.part b/e2e_test/batch/duckdb/cte/test_cte.test.slt.part index 4e073e8ebcacb..68e7abd2ec443 100644 --- a/e2e_test/batch/duckdb/cte/test_cte.test.slt.part +++ b/e2e_test/batch/duckdb/cte/test_cte.test.slt.part @@ -125,7 +125,7 @@ SELECT 1 UNION ALL (WITH cte AS (SELECT 42) SELECT * FROM cte); # test CTE with nested aliases in where clause # Note: postgres doesn't support this: column "alias1" does not exist -query error failed to bind expression: alias1 +query error Failed to bind expression: alias1 with cte (a) as ( select 1 ) diff --git a/e2e_test/ddl/table/generated_columns.slt.part b/e2e_test/ddl/table/generated_columns.slt.part index 0ba892b994865..88294172ac206 100644 --- a/e2e_test/ddl/table/generated_columns.slt.part +++ b/e2e_test/ddl/table/generated_columns.slt.part @@ -140,7 +140,7 @@ statement ok drop source t4; # create a table with generated column now -statement error failed to bind expression: now() +statement error Failed to bind expression: now() CREATE TABLE t (v INT, t timestamptz as now()) WITH ( connector = 'datagen', fields.v.kind = 'sequence', diff --git a/e2e_test/error_ui/main.slt b/e2e_test/error_ui/main.slt index 6dda0023a7827..bdf1c26e1ace6 100644 --- a/e2e_test/error_ui/main.slt +++ b/e2e_test/error_ui/main.slt @@ -25,11 +25,9 @@ select v1 + v2 = v3; ---- db error: ERROR: QueryError -Caused by this error: - 1: Bind error: failed to bind expression: v1 + v2 = v3 - -Caused by: - Item not found: Invalid column: v1 +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: v1 + v2 = v3 + 2: Item not found: Invalid column: v1 query error diff --git a/src/common/src/error.rs b/src/common/src/error.rs index 160ee1a2c61a6..1c59473063b7d 100644 --- a/src/common/src/error.rs +++ b/src/common/src/error.rs @@ -129,8 +129,17 @@ pub enum ErrorCode { // `tonic::transport::Error`, `TonicStatusWrapper`, or `RpcError` BoxedError, ), + // TODO: use a new type for bind error #[error("Bind error: {0}")] BindError(String), + // TODO: only keep this one + #[error("Failed to bind expression: {expr}: {error}")] + BindErrorRoot { + expr: String, + #[source] + #[backtrace] + error: BoxedError, + }, #[error("Catalog error: {0}")] CatalogError( #[source] diff --git a/src/frontend/planner_test/Cargo.toml b/src/frontend/planner_test/Cargo.toml index d19a08bcc2522..47d6ba3a68289 100644 --- a/src/frontend/planner_test/Cargo.toml +++ b/src/frontend/planner_test/Cargo.toml @@ -24,6 +24,7 @@ risingwave_sqlparser = { workspace = true } serde = { version = "1", features = ["derive"] } serde_with = "3" serde_yaml = "0.9" +thiserror-ext = { workspace = true } tokio = { version = "0.2", package = "madsim-tokio", features = [ "rt", "rt-multi-thread", diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index 3e6ebc7ef4322..ea0f15bf78bb2 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -42,6 +42,7 @@ use risingwave_sqlparser::ast::{ }; use risingwave_sqlparser::parser::Parser; use serde::{Deserialize, Serialize}; +use thiserror_ext::AsReport; #[derive(Clone, Debug, PartialEq, Serialize, Deserialize, Hash, Eq)] #[serde(deny_unknown_fields, rename_all = "snake_case")] @@ -578,7 +579,7 @@ impl TestCase { match binder.bind(stmt.clone()) { Ok(bound) => bound, Err(err) => { - ret.binder_error = Some(err.to_string()); + ret.binder_error = Some(err.to_report_string_pretty()); return Ok(ret); } } @@ -594,7 +595,7 @@ impl TestCase { logical_plan } Err(err) => { - ret.planner_error = Some(err.to_string()); + ret.planner_error = Some(err.to_report_string_pretty()); return Ok(ret); } }; @@ -608,7 +609,7 @@ impl TestCase { match logical_plan.gen_optimized_logical_plan_for_batch() { Ok(optimized_logical_plan_for_batch) => optimized_logical_plan_for_batch, Err(err) => { - ret.optimizer_error = Some(err.to_string()); + ret.optimizer_error = Some(err.to_report_string_pretty()); return Ok(ret); } }; @@ -632,7 +633,7 @@ impl TestCase { match logical_plan.gen_optimized_logical_plan_for_stream() { Ok(optimized_logical_plan_for_stream) => optimized_logical_plan_for_stream, Err(err) => { - ret.optimizer_error = Some(err.to_string()); + ret.optimizer_error = Some(err.to_report_string_pretty()); return Ok(ret); } }; @@ -659,12 +660,12 @@ impl TestCase { Ok(batch_plan) => match logical_plan.gen_batch_distributed_plan(batch_plan) { Ok(batch_plan) => batch_plan, Err(err) => { - ret.batch_error = Some(err.to_string()); + ret.batch_error = Some(err.to_report_string_pretty()); break 'batch; } }, Err(err) => { - ret.batch_error = Some(err.to_string()); + ret.batch_error = Some(err.to_report_string_pretty()); break 'batch; } }; @@ -691,12 +692,12 @@ impl TestCase { Ok(batch_plan) => match logical_plan.gen_batch_local_plan(batch_plan) { Ok(batch_plan) => batch_plan, Err(err) => { - ret.batch_error = Some(err.to_string()); + ret.batch_error = Some(err.to_report_string_pretty()); break 'local_batch; } }, Err(err) => { - ret.batch_error = Some(err.to_string()); + ret.batch_error = Some(err.to_report_string_pretty()); break 'local_batch; } }; @@ -759,7 +760,7 @@ impl TestCase { ) { Ok((stream_plan, _)) => stream_plan, Err(err) => { - *ret_error_str = Some(err.to_string()); + *ret_error_str = Some(err.to_report_string_pretty()); continue; } }; @@ -799,7 +800,7 @@ impl TestCase { break 'sink; } Err(err) => { - ret.sink_error = Some(err.to_string()); + ret.sink_error = Some(err.to_report_string_pretty()); break 'sink; } } diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index f1f18c43b2a70..c8de408e7bff0 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1,37 +1,37 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: | values(sum(1)); - binder_error: |- - Bind error: failed to bind expression: sum(1) + binder_error: | + Failed to bind expression: sum(1) Caused by: Invalid input syntax: aggregate functions are not allowed in VALUES - sql: | values(count(1)); - binder_error: |- - Bind error: failed to bind expression: count(1) + binder_error: | + Failed to bind expression: count(1) Caused by: Invalid input syntax: aggregate functions are not allowed in VALUES - sql: | values(min(1)); - binder_error: |- - Bind error: failed to bind expression: min(1) + binder_error: | + Failed to bind expression: min(1) Caused by: Invalid input syntax: aggregate functions are not allowed in VALUES - sql: | values(1 + max(1)); - binder_error: |- - Bind error: failed to bind expression: 1 + max(1) + binder_error: | + Failed to bind expression: 1 + max(1) Caused by: Invalid input syntax: aggregate functions are not allowed in VALUES - sql: | create table t (v1 int); select v1 from t where min(v1); - binder_error: |- - Bind error: failed to bind expression: min(v1) + binder_error: | + Failed to bind expression: min(v1) Caused by: Invalid input syntax: aggregate functions are not allowed in WHERE @@ -233,8 +233,8 @@ - name: group by output column name expr disallowed sql: | select 4 + 5 as a group by a + 1; - binder_error: |- - Bind error: failed to bind expression: a + 1 + binder_error: | + Failed to bind expression: a + 1 Caused by: Item not found: Invalid column: a @@ -252,8 +252,8 @@ - name: group by column not found sql: | select 4 + 5 as a group by b; - binder_error: |- - Bind error: failed to bind expression: b + binder_error: | + Failed to bind expression: b Caused by: Item not found: Invalid column: b @@ -740,8 +740,8 @@ sql: | create table t(a int, b int); select avg(a) FILTER (WHERE abs(a)) AS avga from t; - binder_error: |- - Bind error: failed to bind expression: avg(a) FILTER(WHERE abs(a)) + binder_error: | + Failed to bind expression: avg(a) FILTER(WHERE abs(a)) Caused by: argument of FILTER must be boolean, not type Int32 @@ -749,8 +749,8 @@ sql: | create table t(a int, b int); select avg(a) FILTER (WHERE 0 < (select max(a) from t)) AS avga from t; - binder_error: |- - Bind error: failed to bind expression: avg(a) FILTER(WHERE 0 < (SELECT max(a) FROM t)) + binder_error: | + Failed to bind expression: avg(a) FILTER(WHERE 0 < (SELECT max(a) FROM t)) Caused by: Feature is not yet implemented: subquery in filter clause @@ -759,8 +759,8 @@ sql: | create table t(a int, b int); select avg(a) FILTER (WHERE a < avg(b)) AS avga from t; - binder_error: |- - Bind error: failed to bind expression: avg(a) FILTER(WHERE a < avg(b)) + binder_error: | + Failed to bind expression: avg(a) FILTER(WHERE a < avg(b)) Caused by: Feature is not yet implemented: aggregation function in filter clause @@ -769,8 +769,8 @@ sql: | create table t(a int, b int); select abs(a) FILTER (WHERE a > 0) AS avga from t; - binder_error: |- - Bind error: failed to bind expression: abs(a) FILTER(WHERE a > 0) + binder_error: | + Failed to bind expression: abs(a) FILTER(WHERE a > 0) Caused by: Invalid input syntax: DISTINCT, ORDER BY or FILTER is only allowed in aggregation functions, but `abs` is not an aggregation function @@ -794,8 +794,8 @@ sql: | create table t(x int, y varchar); select count(distinct *) from t; - binder_error: |- - Bind error: failed to bind expression: count(DISTINCT *) + binder_error: | + Failed to bind expression: count(DISTINCT *) Caused by: Invalid input syntax: DISTINCT is not allowed for aggregate function `count` without args @@ -919,8 +919,8 @@ sql: | create table t(x int, y varchar); select string_agg(distinct y, ',' order by x) from t; - binder_error: |- - Bind error: failed to bind expression: string_agg(DISTINCT y, ',' ORDER BY x) + binder_error: | + Failed to bind expression: string_agg(DISTINCT y, ',' ORDER BY x) Caused by: Invalid input syntax: ORDER BY expressions must match regular arguments of the aggregate for `string_agg` when DISTINCT is provided @@ -928,8 +928,8 @@ sql: | create table t(x int, y varchar); select string_agg(distinct y, x::varchar) from t; - binder_error: |- - Bind error: failed to bind expression: string_agg(DISTINCT y, CAST(x AS CHARACTER VARYING)) + binder_error: | + Failed to bind expression: string_agg(DISTINCT y, CAST(x AS CHARACTER VARYING)) Caused by: Feature is not yet implemented: non-constant arguments other than the first one for DISTINCT aggregation is not supported now @@ -1067,8 +1067,8 @@ /* This case is valid in PostgreSQL */ create table a (a1 int, a2 int); select count(a1) filter (where (select true)) from a; - binder_error: |- - Bind error: failed to bind expression: count(a1) FILTER(WHERE (SELECT true)) + binder_error: | + Failed to bind expression: count(a1) FILTER(WHERE (SELECT true)) Caused by: Feature is not yet implemented: subquery in filter clause @@ -1081,22 +1081,20 @@ select 1 from a having exists( select count(b1) filter (where min(a1) < 3) from b ); - binder_error: |- - Bind error: failed to bind expression: EXISTS (SELECT count(b1) FILTER(WHERE min(a1) < 3) FROM b) + binder_error: | + Failed to bind expression: EXISTS (SELECT count(b1) FILTER(WHERE min(a1) < 3) FROM b) - Caused by: - Bind error: failed to bind expression: count(b1) FILTER(WHERE min(a1) < 3) - - Caused by: - Feature is not yet implemented: aggregation function in filter clause + Caused by these errors (recent errors listed first): + 1: Failed to bind expression: count(b1) FILTER(WHERE min(a1) < 3) + 2: Feature is not yet implemented: aggregation function in filter clause No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml - name: agg filter - table function sql: | /* This case is NOT valid in PostgreSQL */ create table a (a1 int, a2 int); select count(a1) filter (where unnest(array[1]) < 1) from a; - binder_error: |- - Bind error: failed to bind expression: count(a1) FILTER(WHERE unnest(ARRAY[1]) < 1) + binder_error: | + Failed to bind expression: count(a1) FILTER(WHERE unnest(ARRAY[1]) < 1) Caused by: Invalid input syntax: table functions are not allowed in FILTER @@ -1524,8 +1522,8 @@ - sql: | create table t (x int, y int); select percentile_cont(x) within group (order by y) from t; - binder_error: |- - Bind error: failed to bind expression: percentile_cont(x) + binder_error: | + Failed to bind expression: percentile_cont(x) Caused by: Feature is not yet implemented: non-constant direct arguments for ordered-set aggregation is not supported now @@ -1533,32 +1531,32 @@ - sql: | create table t (x int, y int); select percentile_cont('abc') within group (order by y) from t; - binder_error: |- - Bind error: failed to bind expression: percentile_cont('abc') + binder_error: | + Failed to bind expression: percentile_cont('abc') Caused by: Invalid input syntax: direct arg in `percentile_cont` must be castable to float64 - sql: | create table t (x int, y int); select percentile_cont(1.3) within group (order by y) from t; - binder_error: |- - Bind error: failed to bind expression: percentile_cont(1.3) + binder_error: | + Failed to bind expression: percentile_cont(1.3) Caused by: Invalid input syntax: direct arg in `percentile_cont` must between 0.0 and 1.0 - sql: | create table t (x int, y int); select percentile_cont(0, 0) within group (order by y) from t; - binder_error: |- - Bind error: failed to bind expression: percentile_cont(0, 0) + binder_error: | + Failed to bind expression: percentile_cont(0, 0) Caused by: Invalid input syntax: invalid direct args or within group argument for `percentile_cont` aggregation - sql: | create table t (x int, y varchar); select percentile_cont(0) within group (order by y) from t; - binder_error: |- - Bind error: failed to bind expression: percentile_cont(0) + binder_error: | + Failed to bind expression: percentile_cont(0) Caused by: Invalid input syntax: arg in `percentile_cont` must be castable to float64 @@ -1587,8 +1585,8 @@ - sql: | create table t (x int, y varchar); select mode(1) within group (order by y desc) from t; - binder_error: |- - Bind error: failed to bind expression: mode(1) + binder_error: | + Failed to bind expression: mode(1) Caused by: Invalid input syntax: invalid direct args or within group argument for `mode` aggregation diff --git a/src/frontend/planner_test/tests/testdata/output/array.yaml b/src/frontend/planner_test/tests/testdata/output/array.yaml index 31e1bfcb88124..f43fba600f885 100644 --- a/src/frontend/planner_test/tests/testdata/output/array.yaml +++ b/src/frontend/planner_test/tests/testdata/output/array.yaml @@ -24,8 +24,8 @@ └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select ARRAY[]; - binder_error: |- - Bind error: failed to bind expression: ARRAY[] + binder_error: | + Failed to bind expression: ARRAY[] Caused by: Bind error: cannot determine type of empty array @@ -42,8 +42,8 @@ └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select ARRAY[]::int; - binder_error: |- - Bind error: failed to bind expression: CAST(ARRAY[] AS INT) + binder_error: | + Failed to bind expression: CAST(ARRAY[] AS INT) Caused by: Bind error: cannot determine type of empty array @@ -86,22 +86,22 @@ batch_plan: 'BatchValues { rows: [[ARRAY[{233}, {66}]:List(List(Int32))]] }' - sql: | select array_cat(array[233], array[array[array[66]]]); - binder_error: |- - Bind error: failed to bind expression: array_cat(ARRAY[233], ARRAY[ARRAY[ARRAY[66]]]) + binder_error: | + Failed to bind expression: array_cat(ARRAY[233], ARRAY[ARRAY[ARRAY[66]]]) Caused by: Bind error: Cannot concatenate integer[] and integer[][][] - sql: | select array_cat(array[233], 123); - binder_error: |- - Bind error: failed to bind expression: array_cat(ARRAY[233], 123) + binder_error: | + Failed to bind expression: array_cat(ARRAY[233], 123) Caused by: Bind error: Cannot concatenate integer[] and integer - sql: | select array_cat(123, array[233]); - binder_error: |- - Bind error: failed to bind expression: array_cat(123, ARRAY[233]) + binder_error: | + Failed to bind expression: array_cat(123, ARRAY[233]) Caused by: Bind error: Cannot concatenate integer and integer[] @@ -124,8 +124,8 @@ - name: array_cat(unknown, int) -> err sql: | select array_cat('{1}', 2); - binder_error: |- - Bind error: failed to bind expression: array_cat('{1}', 2) + binder_error: | + Failed to bind expression: array_cat('{1}', 2) Caused by: Bind error: Cannot concatenate character varying and integer @@ -137,8 +137,8 @@ batch_plan: 'BatchValues { rows: [[ARRAY[66, 123]:List(Int32)]] }' - sql: | select array_append(123, 234); - binder_error: |- - Bind error: failed to bind expression: array_append(123, 234) + binder_error: | + Failed to bind expression: array_append(123, 234) Caused by: Bind error: Cannot append integer to integer @@ -172,8 +172,8 @@ batch_plan: 'BatchValues { rows: [[ARRAY[123, 66]:List(Int32)]] }' - sql: | select array_prepend(123, 234); - binder_error: |- - Bind error: failed to bind expression: array_prepend(123, 234) + binder_error: | + Failed to bind expression: array_prepend(123, 234) Caused by: Bind error: Cannot prepend integer to integer @@ -194,29 +194,29 @@ └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: array_contains(int[], int) -> bool sql: select array[1,2] @> 2; - binder_error: |- - Bind error: failed to bind expression: ARRAY[1, 2] @> 2 + binder_error: | + Failed to bind expression: ARRAY[1, 2] @> 2 Caused by: Bind error: operator does not exist: integer[] @> integer - name: array_contains(int[], varchar[]) -> bool sql: select array[2,3] @> array['1']; - binder_error: |- - Bind error: failed to bind expression: ARRAY[2, 3] @> ARRAY['1'] + binder_error: | + Failed to bind expression: ARRAY[2, 3] @> ARRAY['1'] Caused by: Bind error: Cannot array_contains unnested type integer to unnested type character varying - name: array_contains(int[][], varchar[][]) -> bool sql: select array[array[1,2]] @> array[array['2','3']]; - binder_error: |- - Bind error: failed to bind expression: ARRAY[ARRAY[1, 2]] @> ARRAY[ARRAY['2', '3']] + binder_error: | + Failed to bind expression: ARRAY[ARRAY[1, 2]] @> ARRAY[ARRAY['2', '3']] Caused by: Bind error: Cannot array_contains unnested type integer to unnested type character varying - name: any contains(null, null) -> bool sql: select '{}' @> '{}'; - binder_error: |- - Bind error: failed to bind expression: '{}' @> '{}' + binder_error: | + Failed to bind expression: '{}' @> '{}' Caused by: Bind error: operator does not exist: unknown @> unknown @@ -292,8 +292,8 @@ - name: compare with different type sql: | select array[1] = array[1.2]; - binder_error: |- - Bind error: failed to bind expression: ARRAY[1] = ARRAY[1.2] + binder_error: | + Failed to bind expression: ARRAY[1] = ARRAY[1.2] Caused by: Bind error: cannot compare integer[] and numeric[] diff --git a/src/frontend/planner_test/tests/testdata/output/basic_query.yaml b/src/frontend/planner_test/tests/testdata/output/basic_query.yaml index 3a45e86b90d1d..ab3edebf429b0 100644 --- a/src/frontend/planner_test/tests/testdata/output/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/output/basic_query.yaml @@ -5,7 +5,11 @@ StreamMaterialize { columns: [*VALUES*_0.column_0, *VALUES*_0.column_1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamValues { rows: [[11:Int32, 22:Int32, 0:Int64], [36:Int32, 44:Int32, 1:Int64]] } - sql: select * from t - binder_error: 'Catalog error: table or source not found: t' + binder_error: | + Catalog error + + Caused by: + table or source not found: t - sql: | create table t (v1 bigint, v2 double precision); select * from t; @@ -148,8 +152,8 @@ - sql: | create table t1 (x int); select * from t1 where EXISTS(select * where t1.x=1); - binder_error: |- - Bind error: failed to bind expression: EXISTS (SELECT * WHERE t1.x = 1) + binder_error: | + Failed to bind expression: EXISTS (SELECT * WHERE t1.x = 1) Caused by: Bind error: SELECT * with no tables specified is not valid @@ -175,8 +179,8 @@ sql: | create table t(v1 int); select v1 from t where exists (values(v1)); - binder_error: |- - Bind error: failed to bind expression: EXISTS (VALUES (v1)) + binder_error: | + Failed to bind expression: EXISTS (VALUES (v1)) Caused by: Feature is not yet implemented: CorrelatedInputRef in VALUES diff --git a/src/frontend/planner_test/tests/testdata/output/concat_op_dispatch.yaml b/src/frontend/planner_test/tests/testdata/output/concat_op_dispatch.yaml index 51e42cae94a45..e2cfaad25898e 100644 --- a/src/frontend/planner_test/tests/testdata/output/concat_op_dispatch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/concat_op_dispatch.yaml @@ -26,8 +26,8 @@ - name: int[] || text -> err sql: | with t(s) as (select '2') select array[1] || s from t; - binder_error: |- - Bind error: failed to bind expression: ARRAY[1] || s + binder_error: | + Failed to bind expression: ARRAY[1] || s Caused by: Bind error: Cannot append character varying to integer[] @@ -50,8 +50,8 @@ - name: int || int -> err sql: | select 1 || 2; - binder_error: |- - Bind error: failed to bind expression: 1 || 2 + binder_error: | + Failed to bind expression: 1 || 2 Caused by: Bind error: operator does not exist: integer || integer diff --git a/src/frontend/planner_test/tests/testdata/output/expr.yaml b/src/frontend/planner_test/tests/testdata/output/expr.yaml index f4e9430a88b62..2074a0410b431 100644 --- a/src/frontend/planner_test/tests/testdata/output/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/expr.yaml @@ -13,8 +13,8 @@ └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | values(must_be_unimplemented_func(1)); - binder_error: |- - Bind error: failed to bind expression: must_be_unimplemented_func(1) + binder_error: | + Failed to bind expression: must_be_unimplemented_func(1) Caused by: Feature is not yet implemented: unsupported function: "must_be_unimplemented_func" @@ -73,8 +73,8 @@ - name: in-list with misaligned types sql: | SELECT true in (3, 1.0, 2); - binder_error: |- - Bind error: failed to bind expression: true IN (3, 1.0, 2) + binder_error: | + Failed to bind expression: true IN (3, 1.0, 2) Caused by: Bind error: types Boolean and Int32 cannot be matched @@ -139,8 +139,8 @@ batch_plan: 'BatchValues { rows: [[42:Decimal]] }' - sql: | values(round(true)); - binder_error: |- - Bind error: failed to bind expression: round(true) + binder_error: | + Failed to bind expression: round(true) Caused by: Feature is not yet implemented: Round[Boolean] @@ -160,8 +160,8 @@ batch_plan: 'BatchValues { rows: [[false:Boolean]] }' - sql: | values(1 not like 1.23); - binder_error: |- - Bind error: failed to bind expression: 1 NOT LIKE 1.23 + binder_error: | + Failed to bind expression: 1 NOT LIKE 1.23 Caused by: Feature is not yet implemented: Like[Int32, Decimal] @@ -204,8 +204,8 @@ sql: | create table t (v1 int); select (case when v1=1 then 1 when v1=2 then 2 else true end) from t; - binder_error: |- - Bind error: failed to bind expression: (CASE WHEN v1 = 1 THEN 1 WHEN v1 = 2 THEN 2 ELSE true END) + binder_error: | + Failed to bind expression: (CASE WHEN v1 = 1 THEN 1 WHEN v1 = 2 THEN 2 ELSE true END) Caused by: Bind error: types Int32 and Boolean cannot be matched @@ -213,8 +213,8 @@ sql: | create table t (v1 int); select (case v1 when 1 then 1 when true then 2 else 0.0 end) from t; - binder_error: |- - Bind error: failed to bind expression: (CASE v1 WHEN 1 THEN 1 WHEN true THEN 2 ELSE 0.0 END) + binder_error: | + Failed to bind expression: (CASE v1 WHEN 1 THEN 1 WHEN true THEN 2 ELSE 0.0 END) Caused by: Feature is not yet implemented: Equal[Int32, Boolean] @@ -233,16 +233,16 @@ - sql: | create table t (v1 int); select nullif(v1, 1, 2) from t; - binder_error: |- - Bind error: failed to bind expression: nullif(v1, 1, 2) + binder_error: | + Failed to bind expression: nullif(v1, 1, 2) Caused by: Bind error: Function `nullif` must contain 2 arguments - sql: | create table t (v1 int); select nullif(v1, true) from t; - binder_error: |- - Bind error: failed to bind expression: nullif(v1, true) + binder_error: | + Failed to bind expression: nullif(v1, true) Caused by: Feature is not yet implemented: Equal[Int32, Boolean] @@ -268,16 +268,16 @@ - sql: | create table t (v1 int); select coalesce() from t; - binder_error: |- - Bind error: failed to bind expression: coalesce() + binder_error: | + Failed to bind expression: coalesce() Caused by: Bind error: Function `coalesce` takes at least 1 arguments (0 given) - sql: | create table t (v1 int); select coalesce(1,true) from t; - binder_error: |- - Bind error: failed to bind expression: coalesce(1, true) + binder_error: | + Failed to bind expression: coalesce(1, true) Caused by: Bind error: types Int32 and Boolean cannot be matched @@ -302,16 +302,16 @@ - sql: | create table t (v1 int); select concat_ws(v1, 1.2) from t; - binder_error: |- - Bind error: failed to bind expression: concat_ws(v1, 1.2) + binder_error: | + Failed to bind expression: concat_ws(v1, 1.2) Caused by: Bind error: cannot cast type "integer" to "character varying" in Implicit context - sql: | create table t (v1 int); select concat_ws() from t; - binder_error: |- - Bind error: failed to bind expression: concat_ws() + binder_error: | + Failed to bind expression: concat_ws() Caused by: Bind error: Function `concat_ws` takes at least 2 arguments (0 given) @@ -336,8 +336,8 @@ - sql: | create table t (v1 int); select concat() from t; - binder_error: |- - Bind error: failed to bind expression: concat() + binder_error: | + Failed to bind expression: concat() Caused by: Bind error: Function `concat` takes at least 1 arguments (0 given) @@ -370,8 +370,8 @@ batch_plan: 'BatchValues { rows: [[''integer[][]'':Varchar]] }' - sql: | select 1 = SOME(1); - binder_error: |- - Bind error: failed to bind expression: 1 = SOME(1) + binder_error: | + Failed to bind expression: 1 = SOME(1) Caused by: Bind error: op SOME/ANY/ALL (array) requires array on right side @@ -402,32 +402,32 @@ └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < SOME(null::varchar[]); - binder_error: |- - Bind error: failed to bind expression: 1 < SOME(CAST(NULL AS CHARACTER VARYING[])) + binder_error: | + Failed to bind expression: 1 < SOME(CAST(NULL AS CHARACTER VARYING[])) Caused by: Feature is not yet implemented: LessThan[Int32, Varchar] Tracking issue: https://github.com/risingwavelabs/risingwave/issues/112 - sql: | select 1 < SOME(null::date); - binder_error: |- - Bind error: failed to bind expression: 1 < SOME(CAST(NULL AS DATE)) + binder_error: | + Failed to bind expression: 1 < SOME(CAST(NULL AS DATE)) Caused by: Bind error: op SOME/ANY/ALL (array) requires array on right side - name: array/struct on left not supported yet 5808 sql: | select array[1] < SOME(null); - binder_error: |- - Bind error: failed to bind expression: ARRAY[1] < SOME(NULL) + binder_error: | + Failed to bind expression: ARRAY[1] < SOME(NULL) Caused by: Bind error: array/struct on left are not supported yet - name: array of array/struct on right not supported yet 5808 sql: | select null < SOME(array[array[1]]); - binder_error: |- - Bind error: failed to bind expression: NULL < SOME(ARRAY[ARRAY[1]]) + binder_error: | + Failed to bind expression: NULL < SOME(ARRAY[ARRAY[1]]) Caused by: Bind error: array of array/struct on right are not supported yet @@ -542,8 +542,8 @@ sql: | create table t (v1 timestamp with time zone, v2 int); select sum(v2) as sum_v2 from t group by v1 >= now(); - stream_error: |- - Bind error: failed to bind expression: v1 >= now() + stream_error: | + Failed to bind expression: v1 >= now() Caused by: Invalid input syntax: For streaming queries, `NOW()` function is only allowed in `WHERE`, `HAVING` and `ON`. Found in clause: Some(GroupBy). Please please refer to https://www.risingwave.dev/docs/current/sql-pattern-temporal-filters/ for more information @@ -551,8 +551,8 @@ sql: | create table t (v1 timestamp with time zone, v2 timestamp with time zone); select now() as n, * from t where v1 >= now(); - stream_error: |- - Bind error: failed to bind expression: now() + stream_error: | + Failed to bind expression: now() Caused by: Invalid input syntax: For streaming queries, `NOW()` function is only allowed in `WHERE`, `HAVING` and `ON`. Found in clause: None. Please please refer to https://www.risingwave.dev/docs/current/sql-pattern-temporal-filters/ for more information @@ -560,16 +560,16 @@ sql: | create table t (v1 timestamp with time zone, v2 int); select sum(v2) filter (where v1 >= now()) as sum_v2 from t; - stream_error: |- - Bind error: failed to bind expression: sum(v2) FILTER(WHERE v1 >= now()) + stream_error: | + Failed to bind expression: sum(v2) FILTER(WHERE v1 >= now()) Caused by: Invalid input syntax: For streaming queries, `NOW()` function is only allowed in `WHERE`, `HAVING` and `ON`. Found in clause: Some(Filter). Please please refer to https://www.risingwave.dev/docs/current/sql-pattern-temporal-filters/ for more information - name: typo pg_teminate_backend sql: | select pg_teminate_backend(1); - binder_error: |- - Bind error: failed to bind expression: pg_teminate_backend(1) + binder_error: | + Failed to bind expression: pg_teminate_backend(1) Caused by: Feature is not yet implemented: unsupported function "pg_teminate_backend", do you mean "pg_terminate_backend"? @@ -594,7 +594,11 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: const_eval of division by 0 error sql: select 1 / 0 t1; - batch_error: 'Expr error: Division by zero' + batch_error: | + Expr error + + Caused by: + Division by zero - sql: | select * from abs(-1); batch_plan: 'BatchValues { rows: [[1:Int32]] }' diff --git a/src/frontend/planner_test/tests/testdata/output/format.yaml b/src/frontend/planner_test/tests/testdata/output/format.yaml index 1c42a0f9252ac..89b32834c381e 100644 --- a/src/frontend/planner_test/tests/testdata/output/format.yaml +++ b/src/frontend/planner_test/tests/testdata/output/format.yaml @@ -18,17 +18,29 @@ └─BatchScan { table: t1, columns: [t1.v1, t1.v2, t1.v3], distribution: SomeShard } - sql: | SELECT format('Testing %s, %s, %s, %%', 'one', 'two'); - batch_error: 'Expr error: too few arguments for format()' + batch_error: | + Expr error + + Caused by: + too few arguments for format() - sql: | SELECT format('Testing %s, %s, %s, %', 'one', 'two', 'three'); - batch_error: 'Expr error: Parse error: unterminated format() type specifier' + batch_error: | + Expr error + + Caused by: + Parse error: unterminated format() type specifier - sql: | SELECT format('Testing %s, %f, %d, %', 'one', 'two', 'three'); - batch_error: 'Expr error: Parse error: unrecognized format() type specifier "f"' + batch_error: | + Expr error + + Caused by: + Parse error: unrecognized format() type specifier "f" - sql: | SELECT format(); - binder_error: |- - Bind error: failed to bind expression: format() + binder_error: | + Failed to bind expression: format() Caused by: Bind error: Function `format` takes at least 1 arguments (0 given) diff --git a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml index 2fbfb1d8539b4..bb91f726670a9 100644 --- a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml +++ b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml @@ -20,8 +20,8 @@ - name: select proctime() sql: | select proctime(); - binder_error: |- - Bind error: failed to bind expression: proctime() + binder_error: | + Failed to bind expression: proctime() Caused by: Invalid input syntax: Function `PROCTIME()` is only allowed in CREATE TABLE/SOURCE. Is `NOW()` what you want? diff --git a/src/frontend/planner_test/tests/testdata/output/join.yaml b/src/frontend/planner_test/tests/testdata/output/join.yaml index 2ae639511f20b..095288f45418b 100644 --- a/src/frontend/planner_test/tests/testdata/output/join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join.yaml @@ -289,8 +289,8 @@ create table b(b1 int); create table c(c1 int); select * from a, b join c on a1 + b1 = c1; - binder_error: |- - Bind error: failed to bind expression: a1 + b1 = c1 + binder_error: | + Failed to bind expression: a1 + b1 = c1 Caused by: Item not found: Invalid column: a1 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index dad8d469dcba9..9b754512b1da3 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -1067,8 +1067,8 @@ SESSION_END(B.date_time, INTERVAL '10' SECOND) as endtime FROM bid B GROUP BY B.bidder, SESSION(B.date_time, INTERVAL '10' SECOND); - binder_error: |- - Bind error: failed to bind expression: SESSION_START(B.date_time, INTERVAL '10' SECOND) + binder_error: | + Failed to bind expression: SESSION_START(B.date_time, INTERVAL '10' SECOND) Caused by: Feature is not yet implemented: unsupported function: "session_start" diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml index e5643da203e06..6a7d3c0fa04a6 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -948,8 +948,8 @@ SESSION_END(B.date_time, INTERVAL '10' SECOND) as endtime FROM bid B GROUP BY B.bidder, SESSION(B.date_time, INTERVAL '10' SECOND); - binder_error: |- - Bind error: failed to bind expression: SESSION_START(B.date_time, INTERVAL '10' SECOND) + binder_error: | + Failed to bind expression: SESSION_START(B.date_time, INTERVAL '10' SECOND) Caused by: Feature is not yet implemented: unsupported function: "session_start" @@ -965,8 +965,8 @@ TUMBLE_END(B.p_time, INTERVAL '10' SECOND) as endtime FROM (SELECT *, PROCTIME() as p_time FROM bid) B GROUP BY B.bidder, TUMBLE(B.p_time, INTERVAL '10' SECOND); - binder_error: |- - Bind error: failed to bind expression: PROCTIME() + binder_error: | + Failed to bind expression: PROCTIME() Caused by: Invalid input syntax: Function `PROCTIME()` is only allowed in CREATE TABLE/SOURCE. Is `NOW()` what you want? diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml index c6c3ffd4f5ad6..a956ad3b22e3b 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml @@ -909,8 +909,8 @@ SESSION_END(B.date_time, INTERVAL '10' SECOND) as endtime FROM bid B GROUP BY B.bidder, SESSION(B.date_time, INTERVAL '10' SECOND); - binder_error: |- - Bind error: failed to bind expression: SESSION_START(B.date_time, INTERVAL '10' SECOND) + binder_error: | + Failed to bind expression: SESSION_START(B.date_time, INTERVAL '10' SECOND) Caused by: Feature is not yet implemented: unsupported function: "session_start" @@ -926,8 +926,8 @@ TUMBLE_END(B.p_time, INTERVAL '10' SECOND) as endtime FROM (SELECT *, PROCTIME() as p_time FROM bid) B GROUP BY B.bidder, TUMBLE(B.p_time, INTERVAL '10' SECOND); - binder_error: |- - Bind error: failed to bind expression: PROCTIME() + binder_error: | + Failed to bind expression: PROCTIME() Caused by: Invalid input syntax: Function `PROCTIME()` is only allowed in CREATE TABLE/SOURCE. Is `NOW()` what you want? diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml index 827eaf3d37e80..7f41cfffe04f9 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -1167,8 +1167,8 @@ SESSION_END(B.date_time, INTERVAL '10' SECOND) as endtime FROM bid B GROUP BY B.bidder, SESSION(B.date_time, INTERVAL '10' SECOND); - binder_error: |- - Bind error: failed to bind expression: SESSION_START(B.date_time, INTERVAL '10' SECOND) + binder_error: | + Failed to bind expression: SESSION_START(B.date_time, INTERVAL '10' SECOND) Caused by: Feature is not yet implemented: unsupported function: "session_start" @@ -1184,8 +1184,8 @@ TUMBLE_END(B.p_time, INTERVAL '10' SECOND) as endtime FROM (SELECT *, PROCTIME() as p_time FROM bid) B GROUP BY B.bidder, TUMBLE(B.p_time, INTERVAL '10' SECOND); - binder_error: |- - Bind error: failed to bind expression: PROCTIME() + binder_error: | + Failed to bind expression: PROCTIME() Caused by: Invalid input syntax: Function `PROCTIME()` is only allowed in CREATE TABLE/SOURCE. Is `NOW()` what you want? diff --git a/src/frontend/planner_test/tests/testdata/output/order_by.yaml b/src/frontend/planner_test/tests/testdata/output/order_by.yaml index 548259c834f47..df6490c8e44f4 100644 --- a/src/frontend/planner_test/tests/testdata/output/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/output/order_by.yaml @@ -70,8 +70,8 @@ - sql: | create table t (v1 bigint, v2 double precision); select * from t order by v; - binder_error: |- - Bind error: failed to bind expression: v + binder_error: | + Failed to bind expression: v Caused by: Item not found: Invalid column: v @@ -158,8 +158,8 @@ sql: | create table t (x int, y int); select x + y as sum from t order by sum + 1; - binder_error: |- - Bind error: failed to bind expression: sum + 1 + binder_error: | + Failed to bind expression: sum + 1 Caused by: Item not found: Invalid column: sum diff --git a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml index 07fb3c75a0f2b..0f3c3f6d3ca03 100644 --- a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml @@ -22,8 +22,8 @@ sql: | create table t(x int); select x from t group by (row_number(x) over()); - binder_error: |- - Bind error: failed to bind expression: (row_number(x) OVER ()) + binder_error: | + Failed to bind expression: (row_number(x) OVER ()) Caused by: Invalid input syntax: window functions are not allowed in GROUP BY @@ -31,8 +31,8 @@ sql: | create table t(x int); select x from t having (row_number(x) over() > 1); - binder_error: |- - Bind error: failed to bind expression: (row_number(x) OVER () > 1) + binder_error: | + Failed to bind expression: (row_number(x) OVER () > 1) Caused by: Invalid input syntax: window functions are not allowed in HAVING @@ -40,8 +40,8 @@ sql: | create table t(x int); select x from t where (row_number(x) over() > 1); - binder_error: |- - Bind error: failed to bind expression: (row_number(x) OVER () > 1) + binder_error: | + Failed to bind expression: (row_number(x) OVER () > 1) Caused by: Invalid input syntax: window functions are not allowed in WHERE @@ -49,8 +49,8 @@ sql: | create table t(x int); select sum(x) filter (where row_number() over () > 1) from t; - binder_error: |- - Bind error: failed to bind expression: sum(x) FILTER(WHERE row_number() OVER () > 1) + binder_error: | + Failed to bind expression: sum(x) FILTER(WHERE row_number() OVER () > 1) Caused by: Invalid input syntax: window functions are not allowed in FILTER @@ -58,8 +58,8 @@ sql: | create table t(x int); select lag(x) from t; - binder_error: |- - Bind error: failed to bind expression: lag(x) + binder_error: | + Failed to bind expression: lag(x) Caused by: Invalid input syntax: Window function `lag` must have OVER clause @@ -97,8 +97,8 @@ sql: | create table t(x int); select lead(x, x + 1) over() from t; - binder_error: |- - Bind error: failed to bind expression: lead(x, x + 1) OVER () + binder_error: | + Failed to bind expression: lead(x, x + 1) OVER () Caused by: Feature is not yet implemented: non-const `offset` of `lead` function is not supported yet @@ -239,8 +239,8 @@ sql: | create table t(x int, y int); select x, y, min(x) over(PARTITION BY y ROWS UNBOUNDED FOLLOWING) from t; - binder_error: |- - Bind error: failed to bind expression: min(x) OVER (PARTITION BY y ROWS UNBOUNDED FOLLOWING) + binder_error: | + Failed to bind expression: min(x) OVER (PARTITION BY y ROWS UNBOUNDED FOLLOWING) Caused by: Invalid input syntax: window frame bounds `ROWS BETWEEN UNBOUNDED FOLLOWING AND CURRENT ROW` is not valid @@ -248,8 +248,8 @@ sql: | create table t(x int, y int); select x, y, min(x) over(PARTITION BY y ROWS BETWEEN 1 FOLLOWING AND 2 PRECEDING) from t; - binder_error: |- - Bind error: failed to bind expression: min(x) OVER (PARTITION BY y ROWS BETWEEN 1 FOLLOWING AND 2 PRECEDING) + binder_error: | + Failed to bind expression: min(x) OVER (PARTITION BY y ROWS BETWEEN 1 FOLLOWING AND 2 PRECEDING) Caused by: Invalid input syntax: window frame bounds `ROWS BETWEEN 1 FOLLOWING AND 2 PRECEDING` is not valid @@ -257,8 +257,8 @@ sql: | create table t(x int, y int); select x, y, max(x) over(PARTITION BY y ORDER BY x RANGE 100 PRECEDING) from t; - binder_error: |- - Bind error: failed to bind expression: max(x) OVER (PARTITION BY y ORDER BY x RANGE 100 PRECEDING) + binder_error: | + Failed to bind expression: max(x) OVER (PARTITION BY y ORDER BY x RANGE 100 PRECEDING) Caused by: Feature is not yet implemented: window frame in `RANGE` mode is not supported yet @@ -267,8 +267,8 @@ sql: | create table t(x int, y int); select x, y, max(x) over(PARTITION BY y ORDER BY x RANGE BETWEEN 100 PRECEDING and UNBOUNDED FOLLOWING) from t; - binder_error: |- - Bind error: failed to bind expression: max(x) OVER (PARTITION BY y ORDER BY x RANGE BETWEEN 100 PRECEDING AND UNBOUNDED FOLLOWING) + binder_error: | + Failed to bind expression: max(x) OVER (PARTITION BY y ORDER BY x RANGE BETWEEN 100 PRECEDING AND UNBOUNDED FOLLOWING) Caused by: Feature is not yet implemented: window frame in `RANGE` mode is not supported yet diff --git a/src/frontend/planner_test/tests/testdata/output/project_set.yaml b/src/frontend/planner_test/tests/testdata/output/project_set.yaml index 676772d99d72e..cd8c510b90880 100644 --- a/src/frontend/planner_test/tests/testdata/output/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/output/project_set.yaml @@ -195,40 +195,40 @@ - name: set-returning function disallowed in CASE WHEN sql: | with a(a1) as (values (array[2]), (null)) select case when a1 is not null then unnest(a1) end from a; - binder_error: |- - Bind error: failed to bind expression: CASE WHEN a1 IS NOT NULL THEN unnest(a1) END + binder_error: | + Failed to bind expression: CASE WHEN a1 IS NOT NULL THEN unnest(a1) END Caused by: Bind error: table functions are not allowed in CASE - name: set-returning function disallowed in COALESCE sql: | with a(a1) as (values (array[2]), (null)) select coalesce(unnest(a1), 0) from a; - binder_error: |- - Bind error: failed to bind expression: coalesce(unnest(a1), 0) + binder_error: | + Failed to bind expression: coalesce(unnest(a1), 0) Caused by: Bind error: table functions are not allowed in COALESCE - name: set-returning function disallowed in JOIN ON sql: | with a(a1) as (values (array[2]), (null)), b(b1) as (select 2) select a1 from a join b on unnest(a1) = b1; - binder_error: |- - Bind error: failed to bind expression: unnest(a1) = b1 + binder_error: | + Failed to bind expression: unnest(a1) = b1 Caused by: Invalid input syntax: table functions are not allowed in JOIN ON - name: set-returning function disallowed in HAVING sql: | with a(a1) as (values (array[2]), (null)) select count(*) from a group by a1 having unnest(a1) < 5; - binder_error: |- - Bind error: failed to bind expression: unnest(a1) < 5 + binder_error: | + Failed to bind expression: unnest(a1) < 5 Caused by: Invalid input syntax: table functions are not allowed in HAVING - name: set-returning function disallowed in FILTER sql: | with a(a1) as (values (array[2]), (null)) select count(*) filter(where unnest(a1) < 3) from a; - binder_error: |- - Bind error: failed to bind expression: count(*) FILTER(WHERE unnest(a1) < 3) + binder_error: | + Failed to bind expression: count(*) FILTER(WHERE unnest(a1) < 3) Caused by: Invalid input syntax: table functions are not allowed in FILTER diff --git a/src/frontend/planner_test/tests/testdata/output/range_scan.yaml b/src/frontend/planner_test/tests/testdata/output/range_scan.yaml index f95075c8823a8..fb66ddb3f399b 100644 --- a/src/frontend/planner_test/tests/testdata/output/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/output/range_scan.yaml @@ -32,22 +32,38 @@ - create_table_and_mv sql: | SELECT * FROM orders_count_by_user WHERE user_id = 1/0 - batch_error: 'Expr error: Division by zero' + batch_error: | + Expr error + + Caused by: + Division by zero - before: - create_table_and_mv sql: | SELECT * FROM orders_count_by_user WHERE user_id = 2147483647 + 1 - batch_error: 'Expr error: Numeric out of range' + batch_error: | + Expr error + + Caused by: + Numeric out of range - before: - create_table_and_mv sql: | SELECT * FROM orders_count_by_user WHERE user_id = 'a' - batch_error: 'Expr error: Parse error: bigint invalid digit found in string' + batch_error: | + Expr error + + Caused by: + Parse error: bigint invalid digit found in string - before: - create_table_and_mv sql: | SELECT * FROM orders_count_by_user WHERE user_id > 'a' - batch_error: 'Expr error: Parse error: bigint invalid digit found in string' + batch_error: | + Expr error + + Caused by: + Parse error: bigint invalid digit found in string - before: - create_table_and_mv sql: | @@ -129,7 +145,11 @@ - create_table_and_mv sql: | SELECT * FROM orders_count_by_user WHERE user_id in ('42', '43.0') - batch_error: 'Expr error: Parse error: bigint invalid digit found in string' + batch_error: | + Expr error + + Caused by: + Parse error: bigint invalid digit found in string - before: - create_table_and_mv sql: | diff --git a/src/frontend/planner_test/tests/testdata/output/select_except.yaml b/src/frontend/planner_test/tests/testdata/output/select_except.yaml index ffd6da30b90bc..fbf5c070ecf83 100644 --- a/src/frontend/planner_test/tests/testdata/output/select_except.yaml +++ b/src/frontend/planner_test/tests/testdata/output/select_except.yaml @@ -63,8 +63,8 @@ sql: | create table t (v1 int, v2 int, v3 int); select * except (v1, v4) from t; - binder_error: |- - Bind error: failed to bind expression: v4 + binder_error: | + Failed to bind expression: v4 Caused by: Item not found: Invalid column: v4 @@ -77,8 +77,8 @@ sql: | create table t (v1 int, v2 int); select * except (v1) from t t1 cross join t t2 - binder_error: |- - Bind error: failed to bind expression: v1 + binder_error: | + Failed to bind expression: v1 Caused by: internal error: Ambiguous column name: v1 diff --git a/src/frontend/planner_test/tests/testdata/output/struct_field_access.yaml b/src/frontend/planner_test/tests/testdata/output/struct_field_access.yaml index 7a6f7a9c2d476..aaa620dc56ed2 100644 --- a/src/frontend/planner_test/tests/testdata/output/struct_field_access.yaml +++ b/src/frontend/planner_test/tests/testdata/output/struct_field_access.yaml @@ -2,8 +2,8 @@ - sql: | create table t(c STRUCT); select t.c.x from t - binder_error: |- - Bind error: failed to bind expression: t.c.x + binder_error: | + Failed to bind expression: t.c.x Caused by: Item not found: Invalid column: x @@ -49,14 +49,12 @@ create table t1(c STRUCT); create table t2(c STRUCT); select (c).x from t1 cross join t2; - binder_error: |- - Bind error: failed to bind expression: (c).x + binder_error: | + Failed to bind expression: (c).x - Caused by: - Bind error: failed to bind expression: c - - Caused by: - internal error: Ambiguous column name: c + Caused by these errors (recent errors listed first): + 1: Failed to bind expression: c + 2: internal error: Ambiguous column name: c - sql: | create table t1(c STRUCT); create table t2(c STRUCT); diff --git a/src/frontend/planner_test/tests/testdata/output/struct_query.yaml b/src/frontend/planner_test/tests/testdata/output/struct_query.yaml index 2b9f2de1b423b..c981787d56ba4 100644 --- a/src/frontend/planner_test/tests/testdata/output/struct_query.yaml +++ b/src/frontend/planner_test/tests/testdata/output/struct_query.yaml @@ -262,16 +262,16 @@ - sql: | create table t (v1 bigint, v2 double precision); select (v1).v2 from t; - binder_error: |- - Bind error: failed to bind expression: (v1).v2 + binder_error: | + Failed to bind expression: (v1).v2 Caused by: Bind error: column notation .v2 applied to type bigint, which is not a composite type - sql: | create materialized view t as select * from s; select (country + country) from t; - binder_error: |- - Bind error: failed to bind expression: (country + country) + binder_error: | + Failed to bind expression: (country + country) Caused by: Feature is not yet implemented: Add[Struct(StructType { field_names: ["address", "zipcode"], field_types: [Varchar, Varchar] }), Struct(StructType { field_names: ["address", "zipcode"], field_types: [Varchar, Varchar] })] @@ -296,8 +296,8 @@ - sql: | create materialized view t as select * from s; select avg(country) from t; - binder_error: |- - Bind error: failed to bind expression: avg(country) + binder_error: | + Failed to bind expression: avg(country) Caused by: Invalid input syntax: Invalid aggregation: avg(struct
) @@ -321,8 +321,8 @@ - sql: | create materialized view t as select * from s; select (country).city from t; - binder_error: |- - Bind error: failed to bind expression: (country).city + binder_error: | + Failed to bind expression: (country).city Caused by: Bind error: column "city" not found in struct type diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 812471bc5ec2b..eb76926c35938 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -19,8 +19,8 @@ - sql: | create table t (v1 bigint, v2 double precision); select v1 from (select v2, v1 as v3 from t) where v2 > 1; - binder_error: |- - Bind error: failed to bind expression: v1 + binder_error: | + Failed to bind expression: v1 Caused by: Item not found: Invalid column: v1 @@ -89,24 +89,24 @@ - sql: | create table t(x int); select * from t, (select * from t as t2 order by t.x desc) as t3; - binder_error: |- - Bind error: failed to bind expression: t.x + binder_error: | + Failed to bind expression: t.x Caused by: Item not found: Invalid column: x - sql: | create table t(x int); select * from t, (select t.x) as t1; - binder_error: |- - Bind error: failed to bind expression: t.x + binder_error: | + Failed to bind expression: t.x Caused by: Item not found: Invalid column: x - sql: | create table t(x int); select * from t CROSS JOIN (select t.x) as t1; - binder_error: |- - Bind error: failed to bind expression: t.x + binder_error: | + Failed to bind expression: t.x Caused by: Item not found: Invalid column: x @@ -135,14 +135,12 @@ select * from t where v1 = c ) as t0 ); - binder_error: |- - Bind error: failed to bind expression: EXISTS (SELECT * FROM bc, (SELECT * FROM t WHERE v1 = c) AS t0) + binder_error: | + Failed to bind expression: EXISTS (SELECT * FROM bc, (SELECT * FROM t WHERE v1 = c) AS t0) - Caused by: - Bind error: failed to bind expression: v1 = c - - Caused by: - Item not found: Invalid column: c + Caused by these errors (recent errors listed first): + 1: Failed to bind expression: v1 = c + 2: Item not found: Invalid column: c - name: We need to ensure doubly nested reference to a left table is not permitted sql: | create table ab (a int, b int); @@ -153,8 +151,8 @@ select * from t where v1 = a ) as t0 ); - binder_error: |- - Bind error: failed to bind expression: v1 = a + binder_error: | + Failed to bind expression: v1 = a Caused by: Item not found: Invalid column: a diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml index 9c9bbb2551576..976b913e11749 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml @@ -112,8 +112,8 @@ └─LogicalScan { table: t2, columns: [t2.y] } - sql: | select (select 1, 2); - binder_error: |- - Bind error: failed to bind expression: (SELECT 1, 2) + binder_error: | + Failed to bind expression: (SELECT 1, 2) Caused by: Bind error: Subquery must return only one column @@ -209,11 +209,12 @@ └─LogicalValues { rows: [[1:Int32, 2:Int32, 3:Int32], [4:Int32, 5:Int32, 6:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32, *VALUES*_0.column_2:Int32] } } - sql: | select 1 + (select 2 from t); - binder_error: |- - Bind error: failed to bind expression: 1 + (SELECT 2 FROM t) + binder_error: | + Failed to bind expression: 1 + (SELECT 2 FROM t) - Caused by: - Catalog error: table or source not found: t + Caused by these errors (recent errors listed first): + 1: Catalog error + 2: table or source not found: t - sql: | create table t1(x int, y int); create table t2(x int, y int); diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml index 0d393c378ff85..7aa042f7a119e 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml @@ -210,8 +210,8 @@ create table t1(x int, y int); create table t2(x int, y int); select x from t1 where y in (select y, x from t2 where t1.x > t2.x + 1000); - binder_error: |- - Bind error: failed to bind expression: y IN (SELECT y, x FROM t2 WHERE t1.x > t2.x + 1000) + binder_error: | + Failed to bind expression: y IN (SELECT y, x FROM t2 WHERE t1.x > t2.x + 1000) Caused by: Bind error: Subquery must return only one column diff --git a/src/frontend/planner_test/tests/testdata/output/sysinfo_funcs.yaml b/src/frontend/planner_test/tests/testdata/output/sysinfo_funcs.yaml index 4979b38bba89b..e9daf454af3aa 100644 --- a/src/frontend/planner_test/tests/testdata/output/sysinfo_funcs.yaml +++ b/src/frontend/planner_test/tests/testdata/output/sysinfo_funcs.yaml @@ -19,8 +19,8 @@ batch_plan: 'BatchValues { rows: [[null:List(Varchar)]] }' - sql: | select current_schemas(true and false); - binder_error: |- - Bind error: failed to bind expression: current_schemas(true AND false) + binder_error: | + Failed to bind expression: current_schemas(true AND false) Caused by: Feature is not yet implemented: Only boolean literals are supported in `current_schemas`. diff --git a/src/frontend/planner_test/tests/testdata/output/table_primary_key.yaml b/src/frontend/planner_test/tests/testdata/output/table_primary_key.yaml index 5323e5922f59a..818440435445b 100644 --- a/src/frontend/planner_test/tests/testdata/output/table_primary_key.yaml +++ b/src/frontend/planner_test/tests/testdata/output/table_primary_key.yaml @@ -31,8 +31,8 @@ - create_table_t2 sql: | select _row_id from t2; - binder_error: |- - Bind error: failed to bind expression: _row_id + binder_error: | + Failed to bind expression: _row_id Caused by: Item not found: Invalid column: _row_id diff --git a/src/frontend/planner_test/tests/testdata/output/types.yaml b/src/frontend/planner_test/tests/testdata/output/types.yaml index 4f4cff47b8c11..e3faa44e401de 100644 --- a/src/frontend/planner_test/tests/testdata/output/types.yaml +++ b/src/frontend/planner_test/tests/testdata/output/types.yaml @@ -1,7 +1,7 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: values (''::numeric(20)); - binder_error: |- - Bind error: failed to bind expression: CAST('' AS NUMERIC(20)) + binder_error: | + Failed to bind expression: CAST('' AS NUMERIC(20)) Caused by: Feature is not yet implemented: unsupported data type: NUMERIC(20) @@ -17,8 +17,8 @@ - sql: values (''::timestamp with time zone); logical_plan: 'LogicalValues { rows: [['''':Varchar::Timestamptz]], schema: Schema { fields: [*VALUES*_0.column_0:Timestamptz] } }' - sql: values (''::time with time zone); - binder_error: |- - Bind error: failed to bind expression: CAST('' AS TIME WITH TIME ZONE) + binder_error: | + Failed to bind expression: CAST('' AS TIME WITH TIME ZONE) Caused by: Feature is not yet implemented: unsupported data type: TIME WITH TIME ZONE @@ -28,15 +28,15 @@ LogicalProject { exprs: [1:Int16] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: select ''::"iNt2"; - binder_error: |- - Bind error: failed to bind expression: CAST('' AS "iNt2") + binder_error: | + Failed to bind expression: CAST('' AS "iNt2") Caused by: Feature is not yet implemented: unsupported data type: "iNt2" No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml - sql: select ''::public.int2; - binder_error: |- - Bind error: failed to bind expression: CAST('' AS public.int2) + binder_error: | + Failed to bind expression: CAST('' AS public.int2) Caused by: Feature is not yet implemented: unsupported data type: public.int2 diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index eacd5fc95dd65..09f6ec4ef5903 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -34,45 +34,19 @@ mod subquery; mod value; impl Binder { + /// Bind an expression with `bind_expr_inner`, attach the original expression + /// to the error message. + /// + /// This may only be called at the root of the expression tree or when crossing + /// the boundary of a subquery. Otherwise, the source chain might be too deep + /// and confusing to the user. + // TODO(error-handling): use a dedicated error type during binding to make it clear. pub fn bind_expr(&mut self, expr: Expr) -> Result { - // We use a different function instead `map_err` directly in `bind_expr_inner`, because in - // some cases, recursive error messages don't look good. Whole expr-level should be enough - // in most cases. - // - // e.g., too verbose: - // - // ```ignore - // Bind error: failed to bind expression: a1 + b1 = c1 - // - // Caused by: - // Bind error: failed to bind expression: a1 + b1 - // - // Caused by: - // Bind error: failed to bind expression: a1 - // - // Caused by: - // Item not found: Invalid column: a1 - // ``` - // - // confusing message with an unused subexpr, when the expr is rewritten while binding: - // - // ```ignore - // > create table t (v1 int); - // > select (case v1 when 1 then 1 when true then 2 else 0.0 end) from t; - // - // Bind error: failed to bind expression: CASE v1 WHEN 1 THEN 1 WHEN true THEN 2 ELSE 0.0 END - // - // Caused by: - // Bind error: failed to bind expression: v1 = true - // - // Caused by: - // Feature is not yet implemented: Equal[Int32, Boolean] - // ``` self.bind_expr_inner(expr.clone()).map_err(|e| { - RwError::from(ErrorCode::BindError(format!( - "failed to bind expression: {}\n\nCaused by:\n {}", - expr, e - ))) + RwError::from(ErrorCode::BindErrorRoot { + expr: expr.to_string(), + error: Box::new(e), + }) }) } diff --git a/src/utils/pgwire/src/pg_message.rs b/src/utils/pgwire/src/pg_message.rs index c38182db198f2..838edf52e547b 100644 --- a/src/utils/pgwire/src/pg_message.rs +++ b/src/utils/pgwire/src/pg_message.rs @@ -635,7 +635,7 @@ impl<'a> BeMessage<'a> { // 'E' signalizes ErrorResponse messages buf.put_u8(b'E'); // Format the error as a pretty report. - let msg = format!("{:#}", error.as_ref().as_report()); + let msg = error.to_report_string_pretty(); write_err_or_notice(buf, &ErrorOrNoticeMessage::internal_error(&msg))?; } From 0130e21c77991a3af0082fe56b6688cbacd45661 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 10 Nov 2023 15:57:23 +0800 Subject: [PATCH 32/77] fix(ci): rebuild dashboard artifacts on proto changes (#13360) Signed-off-by: Bugen Zhao --- .github/workflows/dashboard_main.yml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/workflows/dashboard_main.yml b/.github/workflows/dashboard_main.yml index ee7e670bf4ec8..cba5161310a37 100644 --- a/.github/workflows/dashboard_main.yml +++ b/.github/workflows/dashboard_main.yml @@ -1,8 +1,9 @@ name: Dashboard (main) on: push: - branches: [ main ] - paths: [ dashboard/** ] + branches: [main] + paths: [dashboard/**, proto/**] + workflow_dispatch: jobs: dashboard-ui-deploy: runs-on: ubuntu-latest @@ -10,7 +11,7 @@ jobs: - uses: actions/checkout@v3 - uses: actions/setup-node@v2 with: - node-version: '18' + node-version: "18" - uses: arduino/setup-protoc@v1 with: version: "3.x" From 22dc384cfb92272e59641f700ad2207fed1855cd Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Fri, 10 Nov 2023 16:04:41 +0800 Subject: [PATCH 33/77] refactor: use `ExecutorParams::info` in stream scan executors (#13347) Signed-off-by: Richard Chien --- src/compute/tests/cdc_tests.rs | 12 ++- .../src/executor/backfill/cdc/cdc_backfill.rs | 25 ++--- .../executor/backfill/no_shuffle_backfill.rs | 19 ++-- src/stream/src/executor/chain.rs | 23 +++-- src/stream/src/executor/flow_control.rs | 1 - src/stream/src/executor/rearranged_chain.rs | 14 +-- src/stream/src/from_proto/mod.rs | 2 +- .../src/from_proto/source/trad_source.rs | 3 +- src/stream/src/from_proto/stream_scan.rs | 94 ++++++------------- 9 files changed, 73 insertions(+), 120 deletions(-) diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index 1cb35691f8e15..a6e515a1ea571 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -45,8 +45,8 @@ use risingwave_stream::executor::test_utils::MockSource; use risingwave_stream::executor::{ default_source_internal_table, expect_first_barrier, ActorContext, Barrier, BoxedExecutor as StreamBoxedExecutor, BoxedMessageStream, CdcBackfillExecutor, Executor, - MaterializeExecutor, Message, Mutation, PkIndices, PkIndicesRef, SourceStateTableHandler, - StreamExecutorError, + ExecutorInfo, MaterializeExecutor, Message, Mutation, PkIndices, PkIndicesRef, + SourceStateTableHandler, StreamExecutorError, }; // mock upstream binlog offset starting from "1.binlog, pos=0" @@ -203,14 +203,18 @@ async fn test_cdc_backfill() -> StreamResult<()> { .await; let actor_id = 0x1a; + let info = ExecutorInfo { + schema: schema.clone(), + pk_indices: vec![0], + identity: "CdcBackfillExecutor".to_string(), + }; let cdc_backfill = CdcBackfillExecutor::new( ActorContext::create(actor_id), + info, external_table, Box::new(mock_offset_executor), vec![0, 1, 2], None, - schema.clone(), - vec![0], Arc::new(StreamingMetrics::unused()), source_state_handler, false, diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index 333a1ad106340..8b796e538f21a 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -45,13 +45,14 @@ use crate::executor::backfill::utils::{ use crate::executor::monitor::StreamingMetrics; use crate::executor::{ expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, Mutation, PkIndices, PkIndicesRef, SourceStateTableHandler, - StreamExecutorError, StreamExecutorResult, + ExecutorInfo, Message, Mutation, PkIndicesRef, SourceStateTableHandler, StreamExecutorError, + StreamExecutorResult, }; use crate::task::{ActorId, CreateMviewProgress}; pub struct CdcBackfillExecutor { actor_ctx: ActorContextRef, + info: ExecutorInfo, /// Upstream external table upstream_table: ExternalStorageTable, @@ -65,8 +66,6 @@ pub struct CdcBackfillExecutor { actor_id: ActorId, - info: ExecutorInfo, - /// State table of the Source executor source_state_handler: SourceStateTableHandler, @@ -83,33 +82,28 @@ impl CdcBackfillExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, + info: ExecutorInfo, upstream_table: ExternalStorageTable, upstream: BoxedExecutor, output_indices: Vec, progress: Option, - schema: Schema, - pk_indices: PkIndices, metrics: Arc, source_state_handler: SourceStateTableHandler, shared_cdc_source: bool, chunk_size: usize, ) -> Self { Self { - info: ExecutorInfo { - schema, - pk_indices, - identity: "CdcBackfillExecutor".to_owned(), - }, + actor_ctx, + info, upstream_table, upstream, output_indices, actor_id: 0, - metrics, - chunk_size, - actor_ctx, source_state_handler, shared_cdc_source, progress, + metrics, + chunk_size, } } @@ -121,6 +115,7 @@ impl CdcBackfillExecutor { let shared_cdc_source = self.shared_cdc_source; let upstream_table_id = self.upstream_table.table_id().table_id; + let upstream_table_schema = self.upstream_table.schema().clone(); let upstream_table_reader = UpstreamTableReader::new(self.upstream_table); let mut upstream = self.upstream.execute(); @@ -178,7 +173,7 @@ impl CdcBackfillExecutor { }; let mut upstream = if shared_cdc_source { - transform_upstream(upstream, &self.info.schema) + transform_upstream(upstream, &upstream_table_schema) .boxed() .peekable() } else { diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 550fab11e9e9e..99ae7e304e1a9 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -41,7 +41,7 @@ use crate::executor::backfill::utils::{ use crate::executor::monitor::StreamingMetrics; use crate::executor::{ expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, - Message, PkIndices, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, }; use crate::task::{ActorId, CreateMviewProgress}; @@ -80,6 +80,8 @@ pub struct BackfillState { /// in the same worker, so that we can read uncommitted data from the upstream table without /// waiting. pub struct BackfillExecutor { + info: ExecutorInfo, + /// Upstream table upstream_table: StorageTable, /// Upstream with the same schema with the upstream table. @@ -96,8 +98,6 @@ pub struct BackfillExecutor { actor_id: ActorId, - info: ExecutorInfo, - metrics: Arc, chunk_size: usize, @@ -109,29 +109,24 @@ where { #[allow(clippy::too_many_arguments)] pub fn new( + info: ExecutorInfo, upstream_table: StorageTable, upstream: BoxedExecutor, state_table: Option>, output_indices: Vec, progress: CreateMviewProgress, - schema: Schema, - pk_indices: PkIndices, metrics: Arc, chunk_size: usize, - executor_id: u64, ) -> Self { + let actor_id = progress.actor_id(); Self { - info: ExecutorInfo { - schema, - pk_indices, - identity: format!("BackfillExecutor {:X}", executor_id), - }, + info, upstream_table, upstream, state_table, output_indices, - actor_id: progress.actor_id(), progress, + actor_id, metrics, chunk_size, } diff --git a/src/stream/src/executor/chain.rs b/src/stream/src/executor/chain.rs index a51c9e95abbb1..46218012d44df 100644 --- a/src/stream/src/executor/chain.rs +++ b/src/stream/src/executor/chain.rs @@ -18,7 +18,6 @@ use risingwave_common::catalog::Schema; use super::error::StreamExecutorError; use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message}; -use crate::executor::PkIndices; use crate::task::{ActorId, CreateMviewProgress}; /// [`ChainExecutor`] is an executor that enables synchronization between the existing stream and @@ -26,6 +25,8 @@ use crate::task::{ActorId, CreateMviewProgress}; /// feature. It pipes new data of existing MVs to newly created MV only all of the old data in the /// existing MVs are dispatched. pub struct ChainExecutor { + info: ExecutorInfo, + snapshot: BoxedExecutor, upstream: BoxedExecutor, @@ -34,27 +35,20 @@ pub struct ChainExecutor { actor_id: ActorId, - info: ExecutorInfo, - /// Only consume upstream messages. upstream_only: bool, } impl ChainExecutor { pub fn new( + info: ExecutorInfo, snapshot: BoxedExecutor, upstream: BoxedExecutor, progress: CreateMviewProgress, - schema: Schema, - pk_indices: PkIndices, upstream_only: bool, ) -> Self { Self { - info: ExecutorInfo { - schema, - pk_indices, - identity: "Chain".into(), - }, + info, snapshot, upstream, actor_id: progress.actor_id(), @@ -142,7 +136,7 @@ mod test { use super::ChainExecutor; use crate::executor::test_utils::MockSource; - use crate::executor::{Barrier, Executor, Message, Mutation, PkIndices}; + use crate::executor::{Barrier, Executor, ExecutorInfo, Message, Mutation, PkIndices}; use crate::task::{CreateMviewProgress, LocalBarrierManager}; #[tokio::test] @@ -185,7 +179,12 @@ mod test { ], )); - let chain = ChainExecutor::new(first, second, progress, schema, PkIndices::new(), false); + let info = ExecutorInfo { + schema, + pk_indices: PkIndices::new(), + identity: "ChainExecutor".to_string(), + }; + let chain = ChainExecutor::new(info, first, second, progress, false); let mut chain = Box::new(chain).execute(); chain.next().await; diff --git a/src/stream/src/executor/flow_control.rs b/src/stream/src/executor/flow_control.rs index 59a46356101cd..a96518d35ad75 100644 --- a/src/stream/src/executor/flow_control.rs +++ b/src/stream/src/executor/flow_control.rs @@ -36,7 +36,6 @@ pub struct FlowControlExecutor { } impl FlowControlExecutor { - #[allow(clippy::too_many_arguments)] pub fn new(input: Box, rate_limit: Option) -> Self { Self { input, rate_limit } } diff --git a/src/stream/src/executor/rearranged_chain.rs b/src/stream/src/executor/rearranged_chain.rs index d2aaae9fd5025..1207fc371f672 100644 --- a/src/stream/src/executor/rearranged_chain.rs +++ b/src/stream/src/executor/rearranged_chain.rs @@ -25,7 +25,6 @@ use super::error::StreamExecutorError; use super::{ expect_first_barrier, Barrier, BoxedExecutor, Executor, ExecutorInfo, Message, MessageStream, }; -use crate::executor::PkIndices; use crate::task::{ActorId, CreateMviewProgress}; /// `ChainExecutor` is an executor that enables synchronization between the existing stream and @@ -36,6 +35,8 @@ use crate::task::{ActorId, CreateMviewProgress}; /// [`RearrangedChainExecutor`] resolves the latency problem when creating MV with a huge amount of /// existing data, by rearranging the barrier from the upstream. Check the design doc for details. pub struct RearrangedChainExecutor { + info: ExecutorInfo, + snapshot: BoxedExecutor, upstream: BoxedExecutor, @@ -43,8 +44,6 @@ pub struct RearrangedChainExecutor { progress: CreateMviewProgress, actor_id: ActorId, - - info: ExecutorInfo, } #[derive(Debug)] @@ -86,18 +85,13 @@ impl RearrangedMessage { impl RearrangedChainExecutor { pub fn new( + info: ExecutorInfo, snapshot: BoxedExecutor, upstream: BoxedExecutor, progress: CreateMviewProgress, - schema: Schema, - pk_indices: PkIndices, ) -> Self { Self { - info: ExecutorInfo { - schema, - pk_indices, - identity: "RearrangedChain".to_owned(), - }, + info, snapshot, upstream, actor_id: progress.actor_id(), diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs index de69e72fbeb1d..35c529234160c 100644 --- a/src/stream/src/from_proto/mod.rs +++ b/src/stream/src/from_proto/mod.rs @@ -139,7 +139,7 @@ pub async fn create_executor( NodeBody::HashAgg => HashAggExecutorBuilder, NodeBody::HashJoin => HashJoinExecutorBuilder, NodeBody::HopWindow => HopWindowExecutorBuilder, - NodeBody::StreamScan => ChainExecutorBuilder, + NodeBody::StreamScan => StreamScanExecutorBuilder, NodeBody::BatchPlan => BatchQueryExecutorBuilder, NodeBody::Merge => MergeExecutorBuilder, NodeBody::Materialize => MaterializeExecutorBuilder, diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 7557d9ed72871..c719a75bc3b4b 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -195,12 +195,11 @@ impl ExecutorBuilder for SourceExecutorBuilder { // use schema from table_desc let cdc_backfill = CdcBackfillExecutor::new( params.actor_context.clone(), + params.info.clone(), external_table, Box::new(source_exec), (0..table_schema.len()).collect_vec(), None, - table_schema, - params.pk_indices, params.executor_stats, source_state_handler, false, diff --git a/src/stream/src/from_proto/stream_scan.rs b/src/stream/src/from_proto/stream_scan.rs index 259fec62fa034..3f002dd81a9cc 100644 --- a/src/stream/src/from_proto/stream_scan.rs +++ b/src/stream/src/from_proto/stream_scan.rs @@ -32,9 +32,9 @@ use crate::executor::{ RearrangedChainExecutor, SourceStateTableHandler, }; -pub struct ChainExecutorBuilder; +pub struct StreamScanExecutorBuilder; -impl ExecutorBuilder for ChainExecutorBuilder { +impl ExecutorBuilder for StreamScanExecutorBuilder { type Node = StreamScanNode; async fn new_boxed_executor( @@ -55,49 +55,21 @@ impl ExecutorBuilder for ChainExecutorBuilder { .map(|&i| i as usize) .collect_vec(); - let schema = if matches!(node.stream_scan_type(), StreamScanType::Backfill) { - Schema::new( - output_indices - .iter() - .map(|i| snapshot.schema().fields()[*i].clone()) - .collect_vec(), - ) - } else if matches!(node.stream_scan_type(), StreamScanType::CdcBackfill) { - let table_desc: &ExternalTableDesc = node.get_cdc_table_desc()?; - let schema = Schema::new(table_desc.columns.iter().map(Into::into).collect()); - assert_eq!(output_indices, (0..schema.len()).collect_vec()); - schema - } else { - // For `Chain`s other than `Backfill`, there should be no extra mapping required. We can - // directly output the columns received from the upstream or snapshot. - let all_indices = (0..snapshot.schema().len()).collect_vec(); - assert_eq!(output_indices, all_indices); - snapshot.schema().clone() - }; - let executor = match node.stream_scan_type() { StreamScanType::Chain | StreamScanType::UpstreamOnly => { let upstream_only = matches!(node.stream_scan_type(), StreamScanType::UpstreamOnly); - ChainExecutor::new( - snapshot, - upstream, - progress, - schema, - params.pk_indices, - upstream_only, - ) - .boxed() + ChainExecutor::new(params.info, snapshot, upstream, progress, upstream_only).boxed() + } + StreamScanType::Rearrange => { + RearrangedChainExecutor::new(params.info, snapshot, upstream, progress).boxed() } - StreamScanType::Rearrange => RearrangedChainExecutor::new( - snapshot, - upstream, - progress, - schema, - params.pk_indices, - ) - .boxed(), StreamScanType::CdcBackfill => { let table_desc: &ExternalTableDesc = node.get_cdc_table_desc()?; + + let table_schema: Schema = table_desc.columns.iter().map(Into::into).collect(); + assert_eq!(output_indices, (0..table_schema.len()).collect_vec()); + assert_eq!(table_schema.data_types(), params.info.schema.data_types()); + let properties: HashMap = table_desc .connect_properties .iter() @@ -105,15 +77,14 @@ impl ExecutorBuilder for ChainExecutorBuilder { .collect(); let table_type = CdcTableType::from_properties(&properties); let table_reader = - table_type.create_table_reader(properties.clone(), schema.clone())?; + table_type.create_table_reader(properties.clone(), table_schema.clone())?; - let order_types = table_desc + let table_pk_order_types = table_desc .pk .iter() .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) .collect_vec(); - - let pk_indices = table_desc + let table_pk_indices = table_desc .pk .iter() .map(|k| k.column_index as usize) @@ -124,10 +95,10 @@ impl ExecutorBuilder for ChainExecutorBuilder { TableId::new(table_desc.table_id), schema_table_name, table_reader, - schema.clone(), - order_types, - pk_indices.clone(), - (0..table_desc.columns.len()).collect_vec(), + table_schema, + table_pk_order_types, + table_pk_indices, + output_indices.clone(), ); let source_state_handler = SourceStateTableHandler::from_table_catalog( @@ -137,17 +108,17 @@ impl ExecutorBuilder for ChainExecutorBuilder { .await; CdcBackfillExecutor::new( params.actor_context.clone(), + params.info, external_table, upstream, - (0..table_desc.columns.len()).collect_vec(), /* eliminate the last column (_rw_offset) */ + output_indices, Some(progress), - schema.clone(), - pk_indices, params.executor_stats, source_state_handler, true, params.env.config().developer.chunk_size, - ).boxed() + ) + .boxed() } StreamScanType::Backfill => { let table_desc: &StorageTableDesc = node @@ -157,12 +128,6 @@ impl ExecutorBuilder for ChainExecutorBuilder { table_id: table_desc.table_id, }; - let order_types = table_desc - .pk - .iter() - .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) - .collect_vec(); - let column_descs = table_desc .columns .iter() @@ -174,8 +139,13 @@ impl ExecutorBuilder for ChainExecutorBuilder { .map(ColumnId::from) .collect_vec(); + let table_pk_order_types = table_desc + .pk + .iter() + .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) + .collect_vec(); // Use indices based on full table instead of streaming executor output. - let pk_indices = table_desc + let table_pk_indices = table_desc .pk .iter() .map(|k| k.column_index as usize) @@ -216,8 +186,8 @@ impl ExecutorBuilder for ChainExecutorBuilder { table_id, column_descs, column_ids, - order_types, - pk_indices, + table_pk_order_types, + table_pk_indices, distribution, table_option, value_indices, @@ -231,16 +201,14 @@ impl ExecutorBuilder for ChainExecutorBuilder { }; BackfillExecutor::new( + params.info, upstream_table, upstream, state_table, output_indices, progress, - schema, - params.pk_indices, stream.streaming_metrics.clone(), params.env.config().developer.chunk_size, - params.executor_id, ) .boxed() } From 26c7865708513170c43844ff4b43315602481ef3 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 10 Nov 2023 16:33:36 +0800 Subject: [PATCH 34/77] fix(meta): singleton requirement should be propagated through no-shuffle edges in scheduler (#13361) Signed-off-by: Bugen Zhao --- e2e_test/streaming/bug_fixes/issue_13094.slt | 29 ++++++++++++++++++++ src/meta/src/stream/stream_graph/schedule.rs | 5 +++- 2 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 e2e_test/streaming/bug_fixes/issue_13094.slt diff --git a/e2e_test/streaming/bug_fixes/issue_13094.slt b/e2e_test/streaming/bug_fixes/issue_13094.slt new file mode 100644 index 0000000000000..23fe6ccfff91a --- /dev/null +++ b/e2e_test/streaming/bug_fixes/issue_13094.slt @@ -0,0 +1,29 @@ +# https://github.com/risingwavelabs/risingwave/issues/13094 +# The singleton requirement should be propagated through no-shuffle edges in the scheduler. + +statement ok +create table t(v int); + +statement ok +create materialized view m as +with sub(v1) as (select max(v) as v1 from t) +select s1.v1 as sv1, s2.v1 as sv2 +from sub as s1, sub as s2 +where s1.v1 = s2.v1; + +statement ok +insert into t values (1); + +statement ok +flush; + +query II +select * from m; +---- +1 1 + +statement ok +drop materialized view m; + +statement ok +drop table t; diff --git a/src/meta/src/stream/stream_graph/schedule.rs b/src/meta/src/stream/stream_graph/schedule.rs index bbde3b6ae9206..f24db98220678 100644 --- a/src/meta/src/stream/stream_graph/schedule.rs +++ b/src/meta/src/stream/stream_graph/schedule.rs @@ -104,7 +104,7 @@ crepe::crepe! { // Requirements from the facts. Requirement(x, d) <- ExternalReq(x, d); - // Requirements of `NoShuffle` edges. + // Requirements propagate through `NoShuffle` edges. Requirement(x, d) <- Edge(x, y, NoShuffle), Requirement(y, d); Requirement(y, d) <- Edge(x, y, NoShuffle), Requirement(x, d); @@ -112,6 +112,9 @@ crepe::crepe! { SingletonReq(y) <- Edge(_, y, Simple); // The downstream fragment of a `CdcTablename` edge must be singleton. SingletonReq(y) <- Edge(_, y, CdcTablename); + // Singleton requirements propagate through `NoShuffle` edges. + SingletonReq(x) <- Edge(x, y, NoShuffle), SingletonReq(y); + SingletonReq(y) <- Edge(x, y, NoShuffle), SingletonReq(x); // Multiple requirements conflict. Failed(x) <- Requirement(x, d1), Requirement(x, d2), (d1 != d2); From 0b9cb1f711a9936297e3ccec454630b189065624 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Fri, 10 Nov 2023 16:38:03 +0800 Subject: [PATCH 35/77] fix(stream): materialize should not compact input when handling conflict (#13351) --- src/stream/src/executor/mview/materialize.rs | 322 +++++++++++-------- 1 file changed, 185 insertions(+), 137 deletions(-) diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index c068a6eba0de1..59bef6ae7df0a 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -13,15 +13,16 @@ // limitations under the License. use std::collections::hash_map::Entry; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::marker::PhantomData; +use std::ops::Deref; use std::sync::Arc; use bytes::Bytes; use enum_as_inner::EnumAsInner; use futures::{stream, StreamExt}; use futures_async_stream::try_stream; -use itertools::{izip, Itertools}; +use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{ColumnDesc, ColumnId, ConflictBehavior, Schema, TableId}; @@ -30,7 +31,6 @@ use risingwave_common::row::{CompactedRow, RowDeserializer}; use risingwave_common::types::DataType; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; -use risingwave_common::util::row_serde::OrderedRowSerde; use risingwave_common::util::sort_util::ColumnOrder; use risingwave_common::util::value_encoding::BasicSerde; use risingwave_pb::catalog::Table; @@ -142,30 +142,51 @@ impl MaterializeExecutor { match self.conflict_behavior { ConflictBehavior::Overwrite | ConflictBehavior::IgnoreConflict => { - // create MaterializeBuffer from chunk - let buffer = MaterializeBuffer::fill_buffer_from_chunk( - chunk, - self.state_table.value_indices(), - self.state_table.pk_indices(), - self.state_table.pk_serde(), - ); - - if buffer.is_empty() { + if chunk.cardinality() == 0 { // empty chunk continue; } + let (data_chunk, ops) = chunk.into_parts(); + + if self.state_table.value_indices().is_some() { + // TODO(st1page): when materialize partial columns(), we should + // construct some columns in the pk + panic!("materialize executor with data check can not handle only materialize partial columns") + }; + let values = data_chunk.serialize(); + + let key_chunk = data_chunk.project(self.state_table.pk_indices()); + + let pks = { + let mut pks = vec![vec![]; data_chunk.capacity()]; + key_chunk + .rows_with_holes() + .zip_eq_fast(pks.iter_mut()) + .for_each(|(r, vnode_and_pk)| { + if let Some(r) = r { + self.state_table.pk_serde().serialize(r, vnode_and_pk); + } + }); + pks + }; + let (_, vis) = key_chunk.into_parts(); + let row_ops = ops + .iter() + .zip_eq_debug(pks.into_iter()) + .zip_eq_debug(values.into_iter()) + .zip_eq_debug(vis.iter()) + .filter_map(|(((op, k), v), vis)| vis.then_some((*op, k, v))) + .collect_vec(); let fixed_changes = self .materialize_cache - .handle_conflict(buffer, &self.state_table, &self.conflict_behavior) + .handle_conflict( + row_ops, + &self.state_table, + &self.conflict_behavior, + ) .await?; - // TODO(st1page): when materialize partial columns(), we should - // construct some columns in the pk - if self.state_table.value_indices().is_some() { - panic!("materialize executor with data check can not handle only materialize partial columns") - } - match generate_output(fixed_changes, data_types.clone())? { Some(output_chunk) => { self.state_table.write_chunk(output_chunk.clone()); @@ -252,7 +273,7 @@ impl MaterializeExecutor { /// Construct output `StreamChunk` from given buffer. fn generate_output( - changes: Vec<(Vec, KeyOp)>, + changes: MaterializeBuffer, data_types: Vec, ) -> StreamExecutorResult> { // construct output chunk @@ -260,7 +281,7 @@ fn generate_output( let mut new_ops: Vec = vec![]; let mut new_rows: Vec = vec![]; let row_deserializer = RowDeserializer::new(data_types.clone()); - for (_, row_op) in changes { + for (_, row_op) in changes.into_parts() { match row_op { KeyOp::Insert(value) => { new_ops.push(Op::Insert); @@ -306,47 +327,7 @@ impl MaterializeBuffer { } } - #[allow(clippy::disallowed_methods)] - fn fill_buffer_from_chunk( - stream_chunk: StreamChunk, - value_indices: &Option>, - pk_indices: &[usize], - pk_serde: &OrderedRowSerde, - ) -> Self { - let (data_chunk, ops) = stream_chunk.into_parts(); - - let values = if let Some(ref value_indices) = value_indices { - data_chunk.project(value_indices).serialize() - } else { - data_chunk.serialize() - }; - - let mut pks = vec![vec![]; data_chunk.capacity()]; - let key_chunk = data_chunk.project(pk_indices); - key_chunk - .rows_with_holes() - .zip_eq_fast(pks.iter_mut()) - .for_each(|(r, vnode_and_pk)| { - if let Some(r) = r { - pk_serde.serialize(r, vnode_and_pk); - } - }); - - let (_, vis) = key_chunk.into_parts(); - - let mut buffer = MaterializeBuffer::new(); - for ((op, key, value), vis) in izip!(ops.iter(), pks, values).zip_eq_debug(vis.iter()) { - if vis { - match op { - Op::Insert | Op::UpdateInsert => buffer.insert(key, value), - Op::Delete | Op::UpdateDelete => buffer.delete(key, value), - }; - } - } - buffer - } - - fn insert(&mut self, pk: Vec, value: Bytes) { + pub fn insert(&mut self, pk: Vec, value: Bytes) { let entry = self.buffer.entry(pk); match entry { Entry::Vacant(e) => { @@ -357,14 +338,14 @@ impl MaterializeBuffer { let old_val = std::mem::take(old_value); e.insert(KeyOp::Update((old_val, value))); } - _ => { - e.insert(KeyOp::Insert(value)); + KeyOp::Insert(_) | KeyOp::Update(_) => { + unreachable!(); } }, } } - fn delete(&mut self, pk: Vec, old_value: Bytes) { + pub fn delete(&mut self, pk: Vec, old_value: Bytes) { let entry = self.buffer.entry(pk); match entry { Entry::Vacant(e) => { @@ -374,19 +355,35 @@ impl MaterializeBuffer { KeyOp::Insert(_) => { e.remove(); } - _ => { - e.insert(KeyOp::Delete(old_value)); + KeyOp::Update((prev, _curr)) => { + let prev = std::mem::take(prev); + e.insert(KeyOp::Delete(prev)); + } + KeyOp::Delete(_) => { + unreachable!(); } }, } } - fn is_empty(&self) -> bool { - self.buffer.is_empty() - } - - fn keys(&self) -> impl Iterator> { - self.buffer.keys() + pub fn update(&mut self, pk: Vec, old_value: Bytes, new_value: Bytes) { + let entry = self.buffer.entry(pk); + match entry { + Entry::Vacant(e) => { + e.insert(KeyOp::Update((old_value, new_value))); + } + Entry::Occupied(mut e) => match e.get_mut() { + KeyOp::Insert(_) => { + e.insert(KeyOp::Insert(new_value)); + } + KeyOp::Update((_prev, curr)) => { + *curr = new_value; + } + KeyOp::Delete(_) => { + unreachable!() + } + }, + } } pub fn into_parts(self) -> HashMap, KeyOp> { @@ -441,7 +438,8 @@ type EmptyValue = (); impl MaterializeCache { pub fn new(watermark_epoch: AtomicU64Ref, metrics_info: MetricsInfo) -> Self { - let cache = new_unbounded(watermark_epoch, metrics_info.clone()); + let cache: ManagedLruCache, CacheValue> = + new_unbounded(watermark_epoch, metrics_info.clone()); Self { data: cache, metrics_info, @@ -451,28 +449,32 @@ impl MaterializeCache { pub async fn handle_conflict<'a, S: StateStore>( &mut self, - buffer: MaterializeBuffer, + row_ops: Vec<(Op, Vec, Bytes)>, table: &StateTableInner, conflict_behavior: &ConflictBehavior, - ) -> StreamExecutorResult, KeyOp)>> { - // fill cache - self.fetch_keys(buffer.keys().map(|v| v.as_ref()), table, conflict_behavior) + ) -> StreamExecutorResult { + let key_set: HashSet> = row_ops + .iter() + .map(|(_, k, _)| k.as_slice().into()) + .collect(); + + self.fetch_keys(key_set.iter().map(|v| v.deref()), table, conflict_behavior) .await?; - let mut fixed_changes = vec![]; - for (key, row_op) in buffer.into_parts() { + let mut fixed_changes = MaterializeBuffer::new(); + for (op, key, value) in row_ops { let mut update_cache = false; - match row_op { - KeyOp::Insert(new_row) => { + match op { + Op::Insert | Op::UpdateInsert => { match conflict_behavior { ConflictBehavior::Overwrite => { match self.force_get(&key).as_overwrite().unwrap() { - Some(old_row) => fixed_changes.push(( + Some(old_row) => fixed_changes.update( key.clone(), - KeyOp::Update((old_row.row.clone(), new_row.clone())), - )), - None => fixed_changes - .push((key.clone(), KeyOp::Insert(new_row.clone()))), + old_row.row.clone(), + value.clone(), + ), + None => fixed_changes.insert(key.clone(), value.clone()), }; update_cache = true; } @@ -480,8 +482,7 @@ impl MaterializeCache { match self.force_get(&key).as_ignore().unwrap() { Some(_) => (), None => { - fixed_changes - .push((key.clone(), KeyOp::Insert(new_row.clone()))); + fixed_changes.insert(key.clone(), value.clone()); update_cache = true; } }; @@ -494,7 +495,7 @@ impl MaterializeCache { ConflictBehavior::Overwrite => { self.data.push( key, - CacheValue::Overwrite(Some(CompactedRow { row: new_row })), + CacheValue::Overwrite(Some(CompactedRow { row: value })), ); } ConflictBehavior::IgnoreConflict => { @@ -504,13 +505,13 @@ impl MaterializeCache { } } } - KeyOp::Delete(_) => { + + Op::Delete | Op::UpdateDelete => { match conflict_behavior { ConflictBehavior::Overwrite => { match self.force_get(&key).as_overwrite().unwrap() { Some(old_row) => { - fixed_changes - .push((key.clone(), KeyOp::Delete(old_row.row.clone()))); + fixed_changes.delete(key.clone(), old_row.row.clone()); } None => (), // delete a nonexistent value }; @@ -532,47 +533,6 @@ impl MaterializeCache { } } } - KeyOp::Update((_, new_row)) => { - match conflict_behavior { - ConflictBehavior::Overwrite => { - match self.force_get(&key).as_overwrite().unwrap() { - Some(old_row) => fixed_changes.push(( - key.clone(), - KeyOp::Update((old_row.row.clone(), new_row.clone())), - )), - None => fixed_changes - .push((key.clone(), KeyOp::Insert(new_row.clone()))), - } - update_cache = true; - } - ConflictBehavior::IgnoreConflict => { - match self.force_get(&key).as_ignore().unwrap() { - Some(_) => (), - None => { - fixed_changes - .push((key.clone(), KeyOp::Insert(new_row.clone()))); - update_cache = true; - } - }; - } - _ => unreachable!(), - }; - - if update_cache { - match conflict_behavior { - ConflictBehavior::Overwrite => { - self.data.push( - key, - CacheValue::Overwrite(Some(CompactedRow { row: new_row })), - ); - } - ConflictBehavior::IgnoreConflict => { - self.data.push(key, CacheValue::Ignore(Some(()))); - } - _ => unreachable!(), - } - } - } } } Ok(fixed_changes) @@ -760,6 +720,94 @@ mod tests { } } + // https://github.com/risingwavelabs/risingwave/issues/13346 + #[tokio::test] + async fn test_upsert_stream() { + // Prepare storage and memtable. + let memory_state_store = MemoryStateStore::new(); + let table_id = TableId::new(1); + // Two columns of int32 type, the first column is PK. + let schema = Schema::new(vec![ + Field::unnamed(DataType::Int32), + Field::unnamed(DataType::Int32), + ]); + let column_ids = vec![0.into(), 1.into()]; + + // test double insert one pk, the latter needs to override the former. + let chunk1 = StreamChunk::from_pretty( + " i i + + 1 1", + ); + + let chunk2 = StreamChunk::from_pretty( + " i i + + 1 2 + - 1 2", + ); + + // Prepare stream executors. + let source = MockSource::with_messages( + schema.clone(), + PkIndices::new(), + vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(3)), + ], + ); + + let order_types = vec![OrderType::ascending()]; + let column_descs = vec![ + ColumnDesc::unnamed(column_ids[0], DataType::Int32), + ColumnDesc::unnamed(column_ids[1], DataType::Int32), + ]; + + let table = StorageTable::for_test( + memory_state_store.clone(), + table_id, + column_descs, + order_types, + vec![0], + vec![0, 1], + ); + + let mut materialize_executor = Box::new( + MaterializeExecutor::for_test( + Box::new(source), + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + 1, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::Overwrite, + ) + .await, + ) + .execute(); + materialize_executor.next().await.transpose().unwrap(); + + materialize_executor.next().await.transpose().unwrap(); + materialize_executor.next().await.transpose().unwrap(); + materialize_executor.next().await.transpose().unwrap(); + + match materialize_executor.next().await.transpose().unwrap() { + Some(Message::Barrier(_)) => { + let row = table + .get_row( + &OwnedRow::new(vec![Some(1_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert!(row.is_none()); + } + _ => unreachable!(), + } + } + #[tokio::test] async fn test_check_insert_conflict() { // Prepare storage and memtable. @@ -1088,7 +1136,7 @@ mod tests { ]); let column_ids = vec![0.into(), 1.into()]; - // test double insert one pk, the latter needs to override the former. + // test double insert one pk, the latter needs to be ignored. let chunk1 = StreamChunk::from_pretty( " i i + 1 3 @@ -1181,7 +1229,7 @@ mod tests { .unwrap(); assert_eq!( row, - Some(OwnedRow::new(vec![Some(1_i32.into()), Some(4_i32.into())])) + Some(OwnedRow::new(vec![Some(1_i32.into()), Some(3_i32.into())])) ); let row = table @@ -1302,7 +1350,7 @@ mod tests { .unwrap(); assert_eq!( row, - Some(OwnedRow::new(vec![Some(8_i32.into()), Some(3_i32.into())])) + Some(OwnedRow::new(vec![Some(8_i32.into()), Some(2_i32.into())])) ); } _ => unreachable!(), From 205fbf9a88b0077725acadd96d1c7faf7b168823 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Fri, 10 Nov 2023 17:58:22 +0800 Subject: [PATCH 36/77] fix: compaction test open enable_test_epoch flag (#13359) Co-authored-by: Noel Kwan --- ci/workflows/main-cron.yml | 1 + src/tests/compaction_test/Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 891cb69ecb98f..6c7ad6f8e042a 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -63,6 +63,7 @@ steps: if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-test") depends_on: - "build" + - "build-other" - "docslt" plugins: - docker-compose#v4.9.0: diff --git a/src/tests/compaction_test/Cargo.toml b/src/tests/compaction_test/Cargo.toml index 8825e6692d6dd..98fe4ce6eedd2 100644 --- a/src/tests/compaction_test/Cargo.toml +++ b/src/tests/compaction_test/Cargo.toml @@ -24,7 +24,7 @@ prometheus = { version = "0.13" } rand = "0.8" risingwave_common = { workspace = true } risingwave_compactor = { workspace = true } -risingwave_hummock_sdk = { workspace = true } +risingwave_hummock_sdk = { workspace = true, features = ["enable_test_epoch"] } risingwave_hummock_test = { workspace = true } risingwave_meta = { workspace = true } risingwave_meta_node = { workspace = true } From 287604ffe7c4b38d1399d5baa478d081871e78f1 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 10 Nov 2023 18:57:46 +0800 Subject: [PATCH 37/77] refactor(frontend): separate cdc table scan from stream table scan (#13332) --- .../tests/testdata/input/create_source.yaml | 20 ++ .../tests/testdata/output/create_source.yaml | 22 ++ src/frontend/src/handler/create_table.rs | 2 +- src/frontend/src/handler/explain.rs | 28 +- .../src/optimizer/plan_node/logical_scan.rs | 9 +- src/frontend/src/optimizer/plan_node/mod.rs | 7 + .../plan_node/stream_cdc_table_scan.rs | 309 ++++++++++++++++++ .../optimizer/plan_node/stream_table_scan.rs | 76 ++--- 8 files changed, 406 insertions(+), 67 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs diff --git a/src/frontend/planner_test/tests/testdata/input/create_source.yaml b/src/frontend/planner_test/tests/testdata/input/create_source.yaml index 0d53634d3e0ac..0496ad1c13b13 100644 --- a/src/frontend/planner_test/tests/testdata/input/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/input/create_source.yaml @@ -33,3 +33,23 @@ ) FORMAT PLAIN ENCODE CSV (delimiter = E'\t', without_header = true); expected_outputs: - explain_output +- id: create_source_with_cdc_backfill + sql: | + create source mysql_mydb with ( + connector = 'mysql-cdc', + hostname = '127.0.0.1', + port = '8306', + username = 'root', + password = '123456', + database.name = 'mydb', + server.id = 5888 + ); + explain create table t1_rw ( + v1 int, + v2 int, + primary key(v1) + ) from mysql_mydb table 'mydb.t1'; + expected_outputs: + - explain_output + with_config_map: + CDC_BACKFILL: 'true' diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index f440a21a906e9..c20af237eadc1 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -39,3 +39,25 @@ └─StreamRowIdGen { row_id_index: 3 } └─StreamDml { columns: [v1, v2, _rw_key, _row_id] } └─StreamSource { source: s0, columns: [v1, v2, _rw_key, _row_id] } +- id: create_source_with_cdc_backfill + sql: | + create source mysql_mydb with ( + connector = 'mysql-cdc', + hostname = '127.0.0.1', + port = '8306', + username = 'root', + password = '123456', + database.name = 'mydb', + server.id = 5888 + ); + explain create table t1_rw ( + v1 int, + v2 int, + primary key(v1) + ) from mysql_mydb table 'mydb.t1'; + explain_output: | + StreamMaterialize { columns: [v1, v2], stream_key: [v1], pk_columns: [v1], pk_conflict: Overwrite } + └─StreamDml { columns: [v1, v2] } + └─StreamCdcTableScan { table: mydb.t1, columns: [v1, v2] } + with_config_map: + CDC_BACKFILL: 'true' diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 9168f12bc992a..cd8c2544bbcbf 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -784,7 +784,7 @@ fn gen_table_plan_inner( } #[allow(clippy::too_many_arguments)] -fn gen_create_table_plan_for_cdc_source( +pub(crate) fn gen_create_table_plan_for_cdc_source( context: OptimizerContextRef, source_name: ObjectName, table_name: ObjectName, diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index fe798f3fa2857..e16e6b83668a4 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -29,6 +29,7 @@ use super::create_table::{ }; use super::query::gen_batch_plan_by_statement; use super::RwPgResponse; +use crate::handler::create_table::gen_create_table_plan_for_cdc_source; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{Convention, Explain}; @@ -61,6 +62,7 @@ async fn do_handle_explain( source_schema, source_watermarks, append_only, + cdc_table_info, .. } => { // TODO(st1page): refacor it @@ -72,8 +74,9 @@ async fn do_handle_explain( None => (None, None), }; let with_options = context.with_options(); - let plan = match check_create_table_with_source(with_options, source_schema)? { - Some(s) => { + let source_schema = check_create_table_with_source(with_options, source_schema)?; + let plan = match (source_schema, cdc_table_info) { + (Some(s), None) => { gen_create_table_plan_with_source( context, name, @@ -87,7 +90,7 @@ async fn do_handle_explain( .await? .0 } - None => { + (None, None) => { gen_create_table_plan( context, name, @@ -99,6 +102,25 @@ async fn do_handle_explain( )? .0 } + + + (None, Some(cdc_table)) => { + gen_create_table_plan_for_cdc_source( + context.into(), + cdc_table.source_name.clone(), + name.clone(), + cdc_table.external_table_name.clone(), + columns, + constraints, + ColumnIdGenerator::new_initial(), + )?.0 + } + (Some(_), Some(_)) => return Err(ErrorCode::NotSupported( + "Data format and encoding format doesn't apply to table created from a CDC source" + .into(), + "Remove the FORMAT and ENCODE specification".into(), + ) + .into()), }; let context = plan.ctx(); if let Some(notice) = notice { diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 403f67e3c3869..4a8d9bd872d89 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -21,7 +21,6 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, TableDesc}; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::util::sort_util::ColumnOrder; -use risingwave_pb::stream_plan::StreamScanType; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::{childless_record, Distill}; @@ -35,7 +34,7 @@ use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ BatchSeqScan, ColumnPruningContext, LogicalFilter, LogicalProject, LogicalValues, - PredicatePushdownContext, RewriteStreamContext, ToStreamContext, + PredicatePushdownContext, RewriteStreamContext, StreamCdcTableScan, ToStreamContext, }; use crate::optimizer::property::{Cardinality, Order}; use crate::optimizer::rule::IndexSelectionRule; @@ -566,11 +565,7 @@ impl ToStream for LogicalScan { } if self.predicate().always_true() { if self.is_cdc_table() { - Ok(StreamTableScan::new_with_stream_scan_type( - self.core.clone(), - StreamScanType::CdcBackfill, - ) - .into()) + Ok(StreamCdcTableScan::new(self.core.clone()).into()) } else { Ok(StreamTableScan::new(self.core.clone()).into()) } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index ec7777e42e737..e048f0a4707de 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -662,6 +662,9 @@ impl dyn PlanNode { if let Some(stream_table_scan) = self.as_stream_table_scan() { return stream_table_scan.adhoc_to_stream_prost(state); } + if let Some(stream_cdc_table_scan) = self.as_stream_cdc_table_scan() { + return stream_cdc_table_scan.adhoc_to_stream_prost(state); + } if let Some(stream_share) = self.as_stream_share() { return stream_share.adhoc_to_stream_prost(state); } @@ -825,6 +828,7 @@ mod stream_values; mod stream_watermark_filter; mod derive; +mod stream_cdc_table_scan; mod stream_share; mod stream_temporal_join; mod stream_union; @@ -880,6 +884,7 @@ pub use logical_topn::LogicalTopN; pub use logical_union::LogicalUnion; pub use logical_update::LogicalUpdate; pub use logical_values::LogicalValues; +pub use stream_cdc_table_scan::StreamCdcTableScan; pub use stream_dedup::StreamDedup; pub use stream_delta_join::StreamDeltaJoin; pub use stream_dml::StreamDml; @@ -987,6 +992,7 @@ macro_rules! for_all_plan_nodes { , { Stream, Project } , { Stream, Filter } , { Stream, TableScan } + , { Stream, CdcTableScan } , { Stream, Sink } , { Stream, Source } , { Stream, HashJoin } @@ -1097,6 +1103,7 @@ macro_rules! for_stream_plan_nodes { , { Stream, HashJoin } , { Stream, Exchange } , { Stream, TableScan } + , { Stream, CdcTableScan } , { Stream, Sink } , { Stream, Source } , { Stream, HashAgg } diff --git a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs new file mode 100644 index 0000000000000..07284b6cf5436 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs @@ -0,0 +1,309 @@ +// 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 itertools::Itertools; +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::{ColumnCatalog, Field}; +use risingwave_common::hash::VirtualNode; +use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::OrderType; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use risingwave_pb::stream_plan::{PbStreamNode, StreamScanType}; + +use super::stream::prelude::*; +use super::utils::{childless_record, Distill}; +use super::{generic, ExprRewritable, PlanBase, PlanNodeId, PlanRef, StreamNode}; +use crate::catalog::ColumnId; +use crate::expr::ExprRewriter; +use crate::handler::create_source::debezium_cdc_source_schema; +use crate::optimizer::plan_node::utils::{IndicesDisplay, TableCatalogBuilder}; +use crate::optimizer::property::{Distribution, DistributionDisplay}; +use crate::stream_fragmenter::BuildFragmentGraphState; +use crate::{Explain, TableCatalog}; + +/// `StreamCdcTableScan` is a virtual plan node to represent a stream cdc table scan. +/// It will be converted to cdc backfill + merge node (for upstream source) +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct StreamCdcTableScan { + pub base: PlanBase, + core: generic::Scan, + batch_plan_id: PlanNodeId, + stream_scan_type: StreamScanType, +} + +impl StreamCdcTableScan { + pub fn new(core: generic::Scan) -> Self { + let batch_plan_id = core.ctx.next_plan_node_id(); + let distribution = Distribution::Single; + let base = PlanBase::new_stream_with_core( + &core, + distribution, + core.append_only(), + false, + core.watermark_columns(), + ); + Self { + base, + core, + batch_plan_id, + stream_scan_type: StreamScanType::CdcBackfill, + } + } + + pub fn table_name(&self) -> &str { + &self.core.table_name + } + + pub fn core(&self) -> &generic::Scan { + &self.core + } + + pub fn stream_scan_type(&self) -> StreamScanType { + StreamScanType::CdcBackfill + } + + /// Build catalog for backfill state + /// + /// Schema: | vnode | pk ... | `backfill_finished` | `row_count` | + /// + /// key: | vnode | + /// value: | pk ... | `backfill_finished` | `row_count` | + /// + /// When we update the backfill progress, + /// we update it for all vnodes. + /// + /// `pk` refers to the upstream pk which we use to track the backfill progress. + /// + /// `vnode` is the corresponding vnode of the upstream's distribution key. + /// It should also match the vnode of the backfill executor. + /// + /// `backfill_finished` is a boolean which just indicates if backfill is done. + /// + /// `row_count` is a count of rows which indicates the # of rows per executor. + /// We used to track this in memory. + /// But for backfill persistence we have to also persist it. + /// + /// FIXME(kwannoel): + /// - Across all vnodes, the values are the same. + /// - e.g. | vnode | pk ... | `backfill_finished` | `row_count` | + /// | 1002 | Int64(1) | t | 10 | + /// | 1003 | Int64(1) | t | 10 | + /// | 1003 | Int64(1) | t | 10 | + /// Eventually we should track progress per vnode, to support scaling with both mview and + /// the corresponding `no_shuffle_backfill`. + /// However this is not high priority, since we are working on supporting arrangement backfill, + /// which already has this capability. + pub fn build_backfill_state_catalog( + &self, + state: &mut BuildFragmentGraphState, + ) -> TableCatalog { + let properties = self.ctx().with_options().internal_table_subset(); + let mut catalog_builder = TableCatalogBuilder::new(properties); + let upstream_schema = &self.core.get_table_columns(); + + // We use vnode as primary key in state table. + // If `Distribution::Single`, vnode will just be `VirtualNode::default()`. + catalog_builder.add_column(&Field::with_name(VirtualNode::RW_TYPE, "vnode")); + catalog_builder.add_order_column(0, OrderType::ascending()); + + // pk columns + for col_order in self.core.primary_key() { + let col = &upstream_schema[col_order.column_index]; + catalog_builder.add_column(&Field::from(col)); + } + + // `backfill_finished` column + catalog_builder.add_column(&Field::with_name( + DataType::Boolean, + format!("{}_backfill_finished", self.table_name()), + )); + + // `row_count` column + catalog_builder.add_column(&Field::with_name( + DataType::Int64, + format!("{}_row_count", self.table_name()), + )); + + // Reuse the state store pk (vnode) as the vnode as well. + catalog_builder.set_vnode_col_idx(0); + catalog_builder.set_dist_key_in_pk(vec![0]); + + let num_of_columns = catalog_builder.columns().len(); + catalog_builder.set_value_indices((1..num_of_columns).collect_vec()); + + catalog_builder + .build(vec![0], 1) + .with_id(state.gen_table_id_wrapped()) + } +} + +impl_plan_tree_node_for_leaf! { StreamCdcTableScan } + +impl Distill for StreamCdcTableScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let verbose = self.base.ctx().is_explain_verbose(); + let mut vec = Vec::with_capacity(4); + vec.push(("table", Pretty::from(self.core.table_name.clone()))); + vec.push(("columns", self.core.columns_pretty(verbose))); + + if verbose { + let pk = IndicesDisplay { + indices: self.stream_key().unwrap_or_default(), + schema: self.base.schema(), + }; + vec.push(("pk", pk.distill())); + let dist = Pretty::display(&DistributionDisplay { + distribution: self.distribution(), + input_schema: self.base.schema(), + }); + vec.push(("dist", dist)); + } + + childless_record("StreamCdcTableScan", vec) + } +} + +impl StreamNode for StreamCdcTableScan { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { + unreachable!("stream scan cannot be converted into a prost body -- call `adhoc_to_stream_prost` instead.") + } +} + +impl StreamCdcTableScan { + pub fn adhoc_to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> PbStreamNode { + use risingwave_pb::stream_plan::*; + + let stream_key = self + .stream_key() + .unwrap_or_else(|| { + panic!( + "should always have a stream key in the stream plan but not, sub plan: {}", + PlanRef::from(self.clone()).explain_to_string() + ) + }) + .iter() + .map(|x| *x as u32) + .collect_vec(); + + // The required columns from the table (both scan and upstream). + let upstream_column_ids = self + .core + .output_and_pk_column_ids() + .iter() + .map(ColumnId::get_id) + .collect_vec(); + + // The schema of the snapshot read stream + let snapshot_schema = upstream_column_ids + .iter() + .map(|&id| { + let col = self + .core + .get_table_columns() + .iter() + .find(|c| c.column_id.get_id() == id) + .unwrap(); + Field::from(col).to_prost() + }) + .collect_vec(); + + // The schema of the shared cdc source upstream is different from snapshot, + // refer to `debezium_cdc_source_schema()` for details. + let upstream_schema = { + let mut columns = debezium_cdc_source_schema(); + columns.push(ColumnCatalog::row_id_column()); + columns + .into_iter() + .map(|c| Field::from(c.column_desc).to_prost()) + .collect_vec() + }; + + let output_indices = self + .core + .output_column_ids() + .iter() + .map(|i| { + upstream_column_ids + .iter() + .position(|&x| x == i.get_id()) + .unwrap() as u32 + }) + .collect_vec(); + + let batch_plan_node = BatchPlanNode { + table_desc: None, + column_ids: upstream_column_ids.clone(), + }; + + let catalog = self + .build_backfill_state_catalog(state) + .to_internal_table_prost(); + + let node_body = + // don't need batch plan for cdc source + PbNodeBody::StreamScan(StreamScanNode { + table_id: self.core.cdc_table_desc.table_id.table_id, + stream_scan_type: self.stream_scan_type as i32, + // The column indices need to be forwarded to the downstream + output_indices, + upstream_column_ids, + // The table desc used by backfill executor + state_table: Some(catalog), + rate_limit: None, + cdc_table_desc: Some(self.core.cdc_table_desc.to_protobuf()), + ..Default::default() + }); + + PbStreamNode { + fields: self.schema().to_prost(), + input: vec![ + // The merge node body will be filled by the `ActorBuilder` on the meta service. + PbStreamNode { + node_body: Some(PbNodeBody::Merge(Default::default())), + identity: "Upstream".into(), + fields: upstream_schema.clone(), + stream_key: vec![], // not used + ..Default::default() + }, + PbStreamNode { + node_body: Some(PbNodeBody::BatchPlan(batch_plan_node)), + operator_id: self.batch_plan_id.0 as u64, + identity: "BatchPlanNode".into(), + fields: snapshot_schema, + stream_key: vec![], // not used + input: vec![], + append_only: true, + }, + ], + + node_body: Some(node_body), + stream_key, + operator_id: self.base.id().0 as u64, + identity: self.distill_to_string(), + append_only: self.append_only(), + } + } +} + +impl ExprRewritable for StreamCdcTableScan { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self::new(core).into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 1b6aa8d84aa14..e506c69ffb3b2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -17,7 +17,7 @@ use std::rc::Rc; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::{ColumnCatalog, Field, TableDesc}; +use risingwave_common::catalog::{Field, TableDesc}; use risingwave_common::hash::VirtualNode; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; @@ -29,7 +29,6 @@ use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanNodeId, PlanRef, StreamNode}; use crate::catalog::ColumnId; use crate::expr::{ExprRewriter, FunctionCall}; -use crate::handler::create_source::debezium_cdc_source_schema; use crate::optimizer::plan_node::utils::{IndicesDisplay, TableCatalogBuilder}; use crate::optimizer::property::{Distribution, DistributionDisplay}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -57,7 +56,6 @@ impl StreamTableScan { ) -> Self { let batch_plan_id = core.ctx.next_plan_node_id(); - // TODO: correctly derive the distribution for cdc backfill let distribution = { match core.distribution_key() { Some(distribution_key) => { @@ -242,9 +240,6 @@ impl StreamTableScan { .map(|x| *x as u32) .collect_vec(); - // A flag to mark whether the upstream is a cdc source job - let cdc_upstream = matches!(self.stream_scan_type, StreamScanType::CdcBackfill); - // The required columns from the table (both scan and upstream). let upstream_column_ids = match self.stream_scan_type { // For backfill, we additionally need the primary key columns. @@ -274,18 +269,7 @@ impl StreamTableScan { }) .collect_vec(); - // The schema of the shared cdc source upstream is different from snapshot, - // refer to `debezium_cdc_source_schema()` for details. - let upstream_schema = if cdc_upstream { - let mut columns = debezium_cdc_source_schema(); - columns.push(ColumnCatalog::row_id_column()); - columns - .into_iter() - .map(|c| Field::from(c.column_desc).to_prost()) - .collect_vec() - } else { - snapshot_schema.clone() - }; + let upstream_schema = snapshot_schema.clone(); let output_indices = self .core @@ -300,11 +284,7 @@ impl StreamTableScan { .collect_vec(); let batch_plan_node = BatchPlanNode { - table_desc: if cdc_upstream { - None - } else { - Some(self.core.table_desc.to_protobuf()) - }, + table_desc: Some(self.core.table_desc.to_protobuf()), column_ids: upstream_column_ids.clone(), }; @@ -312,39 +292,23 @@ impl StreamTableScan { .build_backfill_state_catalog(state) .to_internal_table_prost(); - let node_body = if cdc_upstream { - // don't need batch plan for cdc source - PbNodeBody::StreamScan(StreamScanNode { - table_id: self.core.cdc_table_desc.table_id.table_id, - stream_scan_type: self.stream_scan_type as i32, - // The column indices need to be forwarded to the downstream - output_indices, - upstream_column_ids, - // The table desc used by backfill executor - state_table: Some(catalog), - rate_limit: None, - cdc_table_desc: Some(self.core.cdc_table_desc.to_protobuf()), - ..Default::default() - }) - } else { - PbNodeBody::StreamScan(StreamScanNode { - table_id: self.core.table_desc.table_id.table_id, - stream_scan_type: self.stream_scan_type as i32, - // The column indices need to be forwarded to the downstream - output_indices, - upstream_column_ids, - // The table desc used by backfill executor - table_desc: Some(self.core.table_desc.to_protobuf()), - state_table: Some(catalog), - rate_limit: self - .base - .ctx() - .session_ctx() - .config() - .get_streaming_rate_limit(), - ..Default::default() - }) - }; + let node_body = PbNodeBody::StreamScan(StreamScanNode { + table_id: self.core.table_desc.table_id.table_id, + stream_scan_type: self.stream_scan_type as i32, + // The column indices need to be forwarded to the downstream + output_indices, + upstream_column_ids, + // The table desc used by backfill executor + table_desc: Some(self.core.table_desc.to_protobuf()), + state_table: Some(catalog), + rate_limit: self + .base + .ctx() + .session_ctx() + .config() + .get_streaming_rate_limit(), + ..Default::default() + }); PbStreamNode { fields: self.schema().to_prost(), From 0c13ba22d5f5000d915c4694b96a8159b1d99faf Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Fri, 10 Nov 2023 19:53:40 +0800 Subject: [PATCH 38/77] fix(source): informative error on protobuf `build_file_descriptor_set` failure (#13364) --- .../src/parser/protobuf/schema_resolver.rs | 29 ++++++++++++------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/src/connector/src/parser/protobuf/schema_resolver.rs b/src/connector/src/parser/protobuf/schema_resolver.rs index 942ae5333cf86..841c6fee825ad 100644 --- a/src/connector/src/parser/protobuf/schema_resolver.rs +++ b/src/connector/src/parser/protobuf/schema_resolver.rs @@ -15,7 +15,10 @@ use std::iter; use std::path::Path; -use protobuf_native::compiler::{SourceTreeDescriptorDatabase, VirtualSourceTree}; +use itertools::Itertools; +use protobuf_native::compiler::{ + SimpleErrorCollector, SourceTreeDescriptorDatabase, VirtualSourceTree, +}; use protobuf_native::MessageLite; use risingwave_common::error::ErrorCode::{InternalError, ProtocolError}; use risingwave_common::error::{Result, RwError}; @@ -48,16 +51,20 @@ pub(super) async fn compile_file_descriptor_from_schema_registry( subject.schema.content.as_bytes().to_vec(), ); } - let mut db = SourceTreeDescriptorDatabase::new(source_tree.as_mut()); - let fds = db - .as_mut() - .build_file_descriptor_set(&[Path::new(&primary_subject.name)]) - .map_err(|e| { - RwError::from(ProtocolError(format!( - "build_file_descriptor_set failed, {}", - e - ))) - })?; + let mut error_collector = SimpleErrorCollector::new(); + // `db` needs to be dropped before we can iterate on `error_collector`. + let fds = { + let mut db = SourceTreeDescriptorDatabase::new(source_tree.as_mut()); + db.as_mut().record_errors_to(error_collector.as_mut()); + db.as_mut() + .build_file_descriptor_set(&[Path::new(&primary_subject.name)]) + } + .map_err(|_| { + RwError::from(ProtocolError(format!( + "build_file_descriptor_set failed. Errors:\n{}", + error_collector.as_mut().join("\n") + ))) + })?; fds.serialize() .map_err(|_| RwError::from(InternalError("serialize descriptor set failed".to_owned()))) } From eabfebaa20f083fcb3f0e3ec76be2149169df35a Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 10 Nov 2023 19:54:51 +0800 Subject: [PATCH 39/77] feat(meta): report background_ddl progress immediately (#13356) --- e2e_test/background_ddl/basic.slt | 35 ++++++++++++------------------- src/meta/src/barrier/mod.rs | 19 ++++++++++++++++- src/meta/src/barrier/progress.rs | 14 ++++++++----- 3 files changed, 40 insertions(+), 28 deletions(-) diff --git a/e2e_test/background_ddl/basic.slt b/e2e_test/background_ddl/basic.slt index d0fee4917b565..b0be505eb1dd9 100644 --- a/e2e_test/background_ddl/basic.slt +++ b/e2e_test/background_ddl/basic.slt @@ -8,50 +8,44 @@ statement ok CREATE TABLE t (v1 int); statement ok -INSERT INTO t select * from generate_series(1, 200000); +INSERT INTO t select * from generate_series(1, 100000); statement ok FLUSH; statement ok -CREATE MATERIALIZED VIEW m1 as SELECT * FROM t; +SET STREAMING_RATE_LIMIT=10000; statement ok -CREATE MATERIALIZED VIEW m2 as SELECT * FROM t; +CREATE MATERIALIZED VIEW m1 as SELECT * FROM t; statement ok -CREATE MATERIALIZED VIEW m3 as SELECT * FROM t; +CREATE MATERIALIZED VIEW m2 as SELECT * FROM t; -# Disable the flaky check: -# query I -# select count(*) from rw_catalog.rw_ddl_progress; -# ---- -# 3 +query I +select count(*) from rw_catalog.rw_ddl_progress; +---- +2 statement error SELECT * FROM m1; # Meta should always reject duplicate mview. statement error -CREATE MATERIALIZED VIEW m3 as SELECT * FROM t; +CREATE MATERIALIZED VIEW m2 as SELECT * FROM t; -# Wait for background ddl to finish -sleep 30s +statement ok +wait; query I select count(*) from m1; ---- -200000 +100000 query I select count(*) from m2; ---- -200000 - -query I -select count(*) from m3; ----- -200000 +100000 statement ok DROP MATERIALIZED VIEW m1; @@ -59,9 +53,6 @@ DROP MATERIALIZED VIEW m1; statement ok DROP MATERIALIZED VIEW m2; -statement ok -DROP MATERIALIZED VIEW m3; - statement ok DROP TABLE t; diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index acd90dce8f522..c0752bd484055 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::assert_matches::assert_matches; +use std::collections::hash_map::Entry; use std::collections::{HashMap, HashSet, VecDeque}; use std::mem::take; use std::ops::Deref; @@ -28,6 +29,7 @@ use risingwave_common::catalog::TableId; use risingwave_common::system_param::PAUSE_ON_NEXT_BOOTSTRAP_KEY; use risingwave_common::util::tracing::TracingContext; use risingwave_hummock_sdk::{ExtendedSstableInfo, HummockSstableObjectId}; +use risingwave_pb::catalog::table::TableType; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::table_fragments::actor_status::ActorState; @@ -1146,7 +1148,22 @@ impl GlobalBarrierManager { } pub async fn get_ddl_progress(&self) -> Vec { - self.tracker.lock().await.gen_ddl_progress() + let mut ddl_progress = self.tracker.lock().await.gen_ddl_progress(); + // If not in tracker, means the first barrier not collected yet. + // In that case just return progress 0. + for table in self.catalog_manager.list_persisted_creating_tables().await { + if table.table_type != TableType::MaterializedView as i32 { + continue; + } + if let Entry::Vacant(e) = ddl_progress.entry(table.id) { + e.insert(DdlProgress { + id: table.id as u64, + statement: table.definition, + progress: "0.0%".into(), + }); + } + } + ddl_progress.into_values().collect() } } diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 6b84ca238f1d7..ce0dc788424bf 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -311,13 +311,17 @@ impl CreateMviewProgressTracker { } } - pub fn gen_ddl_progress(&self) -> Vec { + pub fn gen_ddl_progress(&self) -> HashMap { self.progress_map .iter() - .map(|(table_id, (x, _))| DdlProgress { - id: table_id.table_id as u64, - statement: x.definition.clone(), - progress: format!("{:.2}%", x.calculate_progress() * 100.0), + .map(|(table_id, (x, _))| { + let table_id = table_id.table_id; + let ddl_progress = DdlProgress { + id: table_id as u64, + statement: x.definition.clone(), + progress: format!("{:.2}%", x.calculate_progress() * 100.0), + }; + (table_id, ddl_progress) }) .collect() } From 8f15ace5dd1ddcb652bf6d535cea4fc81fe937a0 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Fri, 10 Nov 2023 20:27:54 +0800 Subject: [PATCH 40/77] fix: temp disable option check on source options (#13362) --- .../src/schema/schema_registry/client.rs | 9 ++---- src/frontend/src/handler/create_source.rs | 32 ++++++++++++++----- src/frontend/src/handler/create_table.rs | 2 +- 3 files changed, 28 insertions(+), 15 deletions(-) diff --git a/src/connector/src/schema/schema_registry/client.rs b/src/connector/src/schema/schema_registry/client.rs index 6d5778cd367c4..39d7bb5373406 100644 --- a/src/connector/src/schema/schema_registry/client.rs +++ b/src/connector/src/schema/schema_registry/client.rs @@ -25,6 +25,9 @@ use serde::de::DeserializeOwned; use super::util::*; +pub const SCHEMA_REGISTRY_USERNAME: &str = "schema.registry.username"; +pub const SCHEMA_REGISTRY_PASSWORD: &str = "schema.registry.password"; + #[derive(Debug, Clone, Default)] pub struct SchemaRegistryAuth { username: Option, @@ -33,9 +36,6 @@ pub struct SchemaRegistryAuth { impl From<&HashMap> for SchemaRegistryAuth { fn from(props: &HashMap) -> Self { - const SCHEMA_REGISTRY_USERNAME: &str = "schema.registry.username"; - const SCHEMA_REGISTRY_PASSWORD: &str = "schema.registry.password"; - SchemaRegistryAuth { username: props.get(SCHEMA_REGISTRY_USERNAME).cloned(), password: props.get(SCHEMA_REGISTRY_PASSWORD).cloned(), @@ -45,9 +45,6 @@ impl From<&HashMap> for SchemaRegistryAuth { impl From<&BTreeMap> for SchemaRegistryAuth { fn from(props: &BTreeMap) -> Self { - const SCHEMA_REGISTRY_USERNAME: &str = "schema.registry.username"; - const SCHEMA_REGISTRY_PASSWORD: &str = "schema.registry.password"; - SchemaRegistryAuth { username: props.get(SCHEMA_REGISTRY_USERNAME).cloned(), password: props.get(SCHEMA_REGISTRY_PASSWORD).cloned(), diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 9e5f80a9a6edd..d9ff525b5d85e 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -31,7 +31,9 @@ use risingwave_connector::parser::{ schema_to_columns, AvroParserConfig, DebeziumAvroParserConfig, ProtobufParserConfig, SpecificParserConfig, }; -use risingwave_connector::schema::schema_registry::name_strategy_from_str; +use risingwave_connector::schema::schema_registry::{ + name_strategy_from_str, SCHEMA_REGISTRY_PASSWORD, SCHEMA_REGISTRY_USERNAME, +}; use risingwave_connector::source::cdc::{ CDC_SHARING_MODE_KEY, CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY, CITUS_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, @@ -291,6 +293,7 @@ fn get_name_strategy_or_default(name_strategy: Option) -> Result for more information. /// return `(columns, source info)` pub(crate) async fn bind_columns_from_source( + session: &SessionImpl, source_schema: &ConnectorSchema, with_properties: &HashMap, create_cdc_source_job: bool, @@ -567,17 +570,25 @@ pub(crate) async fn bind_columns_from_source( )))); } }; + + { + // fixme: remove this after correctly consuming the two options + options.remove(SCHEMA_REGISTRY_USERNAME); + options.remove(SCHEMA_REGISTRY_PASSWORD); + } + if !options.is_empty() { - return Err(RwError::from(ProtocolError(format!( - "Unknown options for {:?} {:?}: {}", + let err_string = format!( + "Get unknown options for {:?} {:?}: {}", source_schema.format, source_schema.row_encode, options - .iter() - .map(|(k, v)| format!("{}:{}", k, v)) + .keys() + .map(|k| k.to_string()) .collect::>() .join(","), - )))); + ); + session.notice_to_user(err_string); } Ok(res) @@ -1135,8 +1146,13 @@ pub async fn handle_create_source( let create_cdc_source_job = is_cdc_connector(&with_properties) && session.config().get_cdc_backfill(); - let (columns_from_resolve_source, source_info) = - bind_columns_from_source(&source_schema, &with_properties, create_cdc_source_job).await?; + let (columns_from_resolve_source, source_info) = bind_columns_from_source( + &session, + &source_schema, + &with_properties, + create_cdc_source_job, + ) + .await?; let columns_from_sql = bind_sql_columns(&stmt.columns)?; let mut columns = bind_all_columns( diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index cd8c2544bbcbf..5654bc5ed4ed0 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -486,7 +486,7 @@ pub(crate) async fn gen_create_table_plan_with_source( let sql_pk_names = bind_sql_pk_names(&column_defs, &constraints)?; let (columns_from_resolve_source, mut source_info) = - bind_columns_from_source(&source_schema, &properties, false).await?; + bind_columns_from_source(context.session_ctx(), &source_schema, &properties, false).await?; let columns_from_sql = bind_sql_columns(&column_defs)?; let mut columns = bind_all_columns( From 6229d81d5171a5642ddc918b8c9870fc472839a0 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Mon, 13 Nov 2023 11:38:13 +0800 Subject: [PATCH 41/77] chore(deps): bump self_cell for RUSTSEC-2023-0070 (#13372) --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8728de3a0e751..003ecdb11c842 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9010,9 +9010,9 @@ dependencies = [ [[package]] name = "self_cell" -version = "1.0.1" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c309e515543e67811222dbc9e3dd7e1056279b782e1dacffe4242b718734fb6" +checksum = "e388332cd64eb80cd595a00941baf513caffae8dce9cfd0467fc9c66397dade6" [[package]] name = "semver" From 77417eb820bc865e91374b2e3459b39bf4acd765 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Mon, 13 Nov 2023 12:02:18 +0800 Subject: [PATCH 42/77] feat(storage): add metrics for mem table spill (#13316) --- docker/dashboards/risingwave-dev-dashboard.json | 2 +- docker/dashboards/risingwave-user-dashboard.json | 2 +- grafana/risingwave-dev-dashboard.dashboard.py | 10 ++++++++++ grafana/risingwave-dev-dashboard.json | 2 +- grafana/risingwave-user-dashboard.json | 2 +- .../src/hummock/store/local_hummock_storage.rs | 5 +++++ .../src/monitor/hummock_state_store_metrics.rs | 16 ++++++++++++++++ 7 files changed, 35 insertions(+), 4 deletions(-) diff --git a/docker/dashboards/risingwave-dev-dashboard.json b/docker/dashboards/risingwave-dev-dashboard.json index ab2d61b963b23..648ef62c32e22 100644 --- a/docker/dashboards/risingwave-dev-dashboard.json +++ b/docker/dashboards/risingwave-dev-dashboard.json @@ -1 +1 @@ -{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dev Dashboard","editable":true,"gnetId":null,"graphTooltip":0,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from actor id to fragment id","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"actor_info{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Id Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from materialized view table id to it's internal table ids","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"table_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"table_info{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Actor/Table Id Info","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of each type of RisingWave components alive.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":5,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","query":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The memory usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":6,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The CPU usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":7,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cpu usage (total) - {{job}} @ {{instance}}","metric":"","query":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cpu usage (avg per core) - {{job}} @ {{instance}}","metric":"","query":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"RW cluster can configure multiple meta nodes to achieve high availability. One is the leader and the rest are the followers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":8,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(meta_num{job=~\"$job\",instance=~\"$node\"}) by (worker_addr,role)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_addr}} @ {{role}}","metric":"","query":"sum(meta_num{job=~\"$job\",instance=~\"$node\"}) by (worker_addr,role)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Meta Cluster","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Cluster Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":2},"height":null,"hideTimeOverride":false,"id":9,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The rate of successful recovery attempts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":10,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","query":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Recovery Successful Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of failed reocovery attempts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":11,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","query":"sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Failed recovery attempts","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Time spent in a successful recovery attempt","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":12,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency pmax - {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by (le) (rate(recovery_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by (le) (rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency avg","metric":"","query":"sum by (le) (rate(recovery_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by (le) (rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Recovery latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Recovery","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":3},"height":null,"hideTimeOverride":false,"id":13,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":14,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Each query is executed in parallel with a user-defined parallelism. This figure shows the throughput of each parallelism. The throughput of all the parallelism added up is equal to Source Throughput(rows).","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":15,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(partition_input_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","query":"rate(partition_input_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":16,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","query":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Each query is executed in parallel with a user-defined parallelism. This figure shows the throughput of each parallelism. The throughput of all the parallelism added up is equal to Source Throughput(MB/s).","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":17,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","query":"(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"RisingWave ingests barriers periodically to trigger computation and checkpoints. The frequency of barrier can be set by barrier_interval_ms. This metric shows how many rows are ingested between two consecutive barriers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":18,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_rows_per_barrier_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} @ {{instance}}","metric":"","query":"rate(stream_source_rows_per_barrier_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) per barrier","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Monitor each source upstream, 0 means the upstream is not normal, 1 means the source is ready.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":19,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"source_status_is_up{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source_id={{source_id}}, source_name={{source_name}} @ {{instance}}","metric":"","query":"source_status_is_up{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Upstream Status","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Source Split Change Events frequency by source_id and actor_id","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":20,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_split_change_event_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_source_split_change_event_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Split Change Events frequency(events/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Kafka Consumer Lag Size by source_id, partition and actor_id","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":21,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"high_watermark{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}} partition={{partition}}","metric":"","query":"high_watermark{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"latest_message_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}} partition={{partition}} actor_id={{actor_id}}","metric":"","query":"latest_message_id{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kafka Consumer Lag Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":22,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}}","metric":"","query":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":23,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id, actor_id) * on(actor_id) group_left(sink_name) sink_info{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}} - actor {{actor_id}}","metric":"","query":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id, actor_id) * on(actor_id) group_left(sink_name) sink_info{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s) per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":24,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}}","metric":"","query":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":25,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (actor_id, table_id) * on(actor_id, table_id) group_left(table_name) table_info{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}} - actor {{actor_id}}","metric":"","query":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (actor_id, table_id) * on(actor_id, table_id) group_left(table_name) table_info{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s) per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been read from the backfill snapshot","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":26,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_snapshot_read_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_backfill_snapshot_read_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Snapshot Read Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been output from the backfill upstream","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":27,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_upstream_output_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_backfill_upstream_output_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Upstream Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of barriers that have been ingested but not completely processed. This metric reflects the current level of congestion within the system.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":28,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all_barrier","metric":"","query":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"in_flight_barrier_nums{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"in_flight_barrier","metric":"","query":"in_flight_barrier_nums{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The duration between the time point when the scheduled barrier needs to be sent and the time point when the barrier gets actually sent to all the compute nodes. Developers can thus detect any internal congestion.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":29,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_send_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_avg","metric":"","query":"rate(meta_barrier_send_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Send Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The time that the data between two consecutive barriers gets fully processed, i.e. the computation results are made durable into materialized views or sink to external systems. This metric shows to users the freshness of materialized views.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":30,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","query":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":31,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_avg","metric":"","query":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier In-Flight Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":32,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p999 - {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_pmax - {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_avg - {{instance}}","metric":"","query":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Sync Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":33,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_wait_commit_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_avg","metric":"","query":"rate(meta_barrier_wait_commit_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Wait Commit Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of actors that have processed the earliest in-flight barriers per second. This metric helps users to detect potential congestion or stuck in the system.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":34,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_barrier_manager_progress{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","query":"rate(stream_barrier_manager_progress{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Earliest In-Flight Barrier Progress","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":4},"height":null,"hideTimeOverride":false,"id":35,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"We first record the total blocking duration(ns) of output buffer of each actor. It shows how much time it takes an actor to process a message, i.e. a barrier, a watermark or rows of data, on average. Then we divide this duration by 1 second and show it as a percentage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":36,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}->{{downstream_fragment_id}}","metric":"","query":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Blocking Time Ratio (Backpressure)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":37,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_input_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, upstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}<-{{upstream_fragment_id}}","metric":"","query":"avg(rate(stream_actor_input_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, upstream_fragment_id) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":38,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","query":"rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":39,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","query":"rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The operator-level memory usage statistics collected by each LRU cache","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":40,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (table_id, desc)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} desc: {{desc}}","metric":"","query":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (table_id, desc)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_memory_usage{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} actor {{actor_id}} desc: {{desc}}","metric":"","query":"stream_memory_usage{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Memory Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Memory usage aggregated by materialized views","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":41,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized view {{materialized_view_id}}","metric":"","query":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Memory Usage of Materialized Views","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":42,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"temporal join cache miss, table_id {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"temporal join cache miss, table_id {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Temporal Join Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":43,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache hit count - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total cached count - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache hit count - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total cached count - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialize Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":44,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache lookup count - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss count - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache lookup count - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss count - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Over Window Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":45,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"join executor cache miss ratio - - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} fragment {{fragment_id}}","metric":"","query":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n appendonly cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream lookup cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream temporal join cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialize executor cache miss ratio - table {{table_id}} fragment {{fragment_id}} {{instance}}","metric":"","query":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_over_window_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_over_window_cache_lookup_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Over window cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_over_window_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_over_window_cache_lookup_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Miss Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":46,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, fragment_id, wait_side, job)(rate(stream_join_barrier_align_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le,fragment_id,wait_side,job) (rate(stream_join_barrier_align_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","query":"sum by(le, fragment_id, wait_side, job)(rate(stream_join_barrier_align_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le,fragment_id,wait_side,job) (rate(stream_join_barrier_align_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Barrier Align","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":47,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","query":"rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":48,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}} {{side}}","metric":"","query":"sum(rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Match Duration Per Second","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Multiple rows with distinct primary keys may have the same join key. This metric counts the number of join keys in the executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":49,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (fragment_id, side)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}} {{side}}","metric":"","query":"sum(stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (fragment_id, side)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}} {{side}}","metric":"","query":"stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of matched rows on the opposite side","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":50,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, actor_id, table_id) (rate(stream_join_matched_join_keys_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, fragment_id, table_id) (rate(stream_join_matched_join_keys_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","query":"sum by(le, job, actor_id, table_id) (rate(stream_join_matched_join_keys_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, fragment_id, table_id) (rate(stream_join_matched_join_keys_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Matched Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":51,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss - table {{table_id}} actor {{actor_id}}}","metric":"","query":"rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each StreamChunk","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":52,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg cached keys count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg distinct cached keys count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg cached keys count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg distinct cached keys count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of dirty (unflushed) groups in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":53,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Dirty Groups Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The total heap size of dirty (unflushed) groups in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":54,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups heap size | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups heap size | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Dirty Groups Heap Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in each top_n executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":55,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n appendonly cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n cached count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n appendonly cached count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"TopN Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in temporal join executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":56,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal Join cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal Join cached count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Temporal Join Cache Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in lookup executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":57,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lookup cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lookup cached count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lookup Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in over window executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":58,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"over window cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"over window cached count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Over Window Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"When enabled, this metric shows the input throughput of each executor.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":59,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_identity, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_identity}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_identity, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_identity}} actor {{actor_id}}","metric":"","query":"rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The actor-level memory usage statistics reported by TaskLocalAlloc. (Disabled by default)","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":60,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(actor_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","query":"sum(actor_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_memory_usage{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","query":"actor_memory_usage{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Memory Usage (TaskLocalAlloc)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":5},"height":null,"hideTimeOverride":false,"id":61,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":62,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_actor_execution_time{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_actor_execution_time{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Execution Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":63,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":64,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":8},"height":null,"hideTimeOverride":false,"id":65,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":8},"height":null,"hideTimeOverride":false,"id":66,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":67,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":16},"height":null,"hideTimeOverride":false,"id":68,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":16},"height":null,"hideTimeOverride":false,"id":69,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":70,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":24},"height":null,"hideTimeOverride":false,"id":71,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":24},"height":null,"hideTimeOverride":false,"id":72,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":73,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":32},"height":null,"hideTimeOverride":false,"id":74,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":32},"height":null,"hideTimeOverride":false,"id":75,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":76,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":40},"height":null,"hideTimeOverride":false,"id":77,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Avg Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors (Tokio)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":6},"height":null,"hideTimeOverride":false,"id":78,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":79,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","query":"rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Send Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":80,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","query":"rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Recv Throughput","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Exchange","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":7},"height":null,"hideTimeOverride":false,"id":81,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":82,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","query":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","query":"sum(user_compute_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compute Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":83,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","query":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","query":"sum(user_source_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":84,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_reader_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: actor_id={{actor_id}}, source_id={{source_id}})","metric":"","query":"sum(user_source_reader_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_reader_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: actor_id={{actor_id}}, source_id={{source_id}})","metric":"","query":"sum(user_source_reader_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Reader Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":85,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_sink_error{job=~\"$job\",instance=~\"$node\"}) by (connector_name, executor_id, error_msg)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{connector_name}}: {{error_msg}} ({{executor_id}})","metric":"","query":"sum(user_sink_error{job=~\"$job\",instance=~\"$node\"}) by (connector_name, executor_id, error_msg)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink by Connector","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"User Streaming Errors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":86,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":87,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","query":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Exchange Recv Row Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":88,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_task_num{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mpp Task Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"All memory usage of batch executors in bytes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":89,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_total_mem{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_total_mem{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mem Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":90,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_heartbeat_worker_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_heartbeat_worker_num{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Heartbeat Worker Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the memory usage of mem_table.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":91,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table size total - {{job}} @ {{instance}}","metric":"","query":"sum(state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table size - table id {{table_id}} instance id {{instance_id}} {{job}} @ {{instance}}","metric":"","query":"state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Mem Table Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the item counts in mem_table.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":92,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table counts total - {{job}} @ {{instance}}","metric":"","query":"sum(state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table count - table id {{table_id}} instance id {{instance_id}} {{job}} @ {{instance}}","metric":"","query":"state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Mem Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":93,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Row SeqScan Next Duration","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":94,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":95,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{table_id}} @ {{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_sst_store_block_request_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total_meta_miss_count - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Hummock has three parts of memory usage: 1. Meta Cache 2. Block CacheThis metric shows the real memory usage of each of these three caches.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":96,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache - {{job}} @ {{instance}}","metric":"","query":"avg(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"data cache - {{job}} @ {{instance}}","metric":"","query":"avg(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":97,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"block cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Miss Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":98,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_scan_key_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type, table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{table_id}} @ {{type}} @ {{instance}}","metric":"","query":"sum(rate(state_store_iter_scan_key_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type, table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Iter keys flow","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":99,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p50 - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p99 - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts pmax - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Merged SSTs","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of the latency of Get operations that have been issued to the state store.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":100,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of the time spent on iterator initialization.Histogram of the time spent on iterator scanning.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":101,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_init_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_init_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_iter_init_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_init_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_scan_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_scan_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_iter_scan_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_scan_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":102,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive count - {{table_id}} - {{type}}","metric":"","query":"sum(irate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter positive count - {{table_id}} - {{type}}","metric":"","query":"sum(irate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter check count- {{table_id}} - {{type}}","metric":"","query":"sum(irate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Positive / Total","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":103,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter positive rate - {{table_id}} - {{type}}","metric":"","query":"(sum(rate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter Positive Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"False-Positive / Total","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":104,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(((sum(rate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter false positive rate - {{table_id}} - {{type}}","metric":"","query":"(((sum(rate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter False-Positive Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":105,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_iter_slow_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"state_store_iter_slow_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Slow Fetch Meta Unhits","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":106,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_shared_buffer_hit_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer hit - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_shared_buffer_hit_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_in_process_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_iter_in_process_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":107,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":108,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":109,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read p50 - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read p99 - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read pmax - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Read Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":110,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Count - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size of a single key-value pair when reading by operation Get.Operation Get gets a single key-value pair with respect to a caller-specified key. If the key does not exist in the storage, the size of key is counted into this metric and the size of value is 0.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":111,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size of all the key-value paris when reading by operation Iter.Operation Iter scans a range of key-value pairs.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":112,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_iter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":113,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":114,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_iter_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"state_store_iter_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Unhits","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock (Read)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":10},"height":null,"hideTimeOverride":false,"id":115,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the real memory usage of uploader.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":116,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading memory - {{job}} @ {{instance}}","metric":"","query":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_uploader_uploading_task_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading task size - {{job}} @ {{instance}}","metric":"","query":"sum(state_store_uploader_uploading_task_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader Memory Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of time spent on compacting shared buffer to remote storage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":117,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(state_store_sync_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Build and Sync Sstable Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":118,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.5, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write p50 - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(0.5, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.99, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write p99 - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(0.99, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write pmax - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Write Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":119,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_merge_imm_task_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"merge imm tasks - {{table_id}} @ {{instance}}","metric":"","query":"sum(irate(state_store_merge_imm_task_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_spill_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Uploader spill tasks - {{uploader_stage}} @ {{instance}}","metric":"","query":"sum(irate(state_store_spill_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader - Tasks Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":120,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_merge_imm_memory_sz{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Merging tasks memory size - {{table_id}} @ {{instance}}","metric":"","query":"sum(rate(state_store_merge_imm_memory_sz{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_spill_task_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Uploading tasks size - {{uploader_stage}} @ {{instance}}","metric":"","query":"sum(rate(state_store_spill_task_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader - Task Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":121,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write batch - {{table_id}} @ {{job}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"l0 - {{job}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":122,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":123,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_write_batch_tuple_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write_batch_kv_pair_count - {{table_id}} @ {{instance}}","metric":"","query":"sum(irate(state_store_write_batch_tuple_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Item Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":124,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_size_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id) / sum(rate(state_store_write_batch_size_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_write_batch_size_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id) / sum(rate(state_store_write_batch_size_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_shared_buffer_to_sstable_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance) / sum(rate(compactor_shared_buffer_to_sstable_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sync - {{job}} @ {{instance}}","metric":"","query":"sum(rate(compactor_shared_buffer_to_sstable_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance) / sum(rate(compactor_shared_buffer_to_sstable_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the statistics of mem_table size on flush. By default only max (p100) is shown.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":125,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, table_id, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, table_id, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_write_batch_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, table_id, job, instance) (rate(state_store_write_batch_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(state_store_write_batch_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, table_id, job, instance) (rate(state_store_write_batch_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Batch Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":126,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Sync Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock (Write)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":11},"height":null,"hideTimeOverride":false,"id":127,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of SSTables at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":128,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"sum(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SSTable Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size(KB) of SSTables at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":129,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SSTable Size(KB)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The of bytes that have been written by commit epoch per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":130,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_commit_write_throughput{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{table_id}}","metric":"","query":"sum(rate(storage_commit_write_throughput{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Commit Flush Bytes by Table","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have completed or failed","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":131,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency{result!='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","query":"sum(storage_level_compact_frequency{result!='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Failure Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have completed or failed","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":132,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency{result='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","query":"sum(storage_level_compact_frequency{result='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Success Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have been skipped.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":133,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_skip_compact_frequency{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (level, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{level}}-{{type}}","metric":"","query":"sum(rate(storage_skip_compact_frequency{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (level, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Skip Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg l0 select_level_count of the compact task, and categorize it according to different cg, levels and task types","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":134,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, group, type)(irate(storage_l0_compact_level_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_l0_compact_level_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg cg{{group}}@{{type}}","metric":"","query":"sum by(le, group, type)(irate(storage_l0_compact_level_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_l0_compact_level_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task L0 Select Level Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg file count of the compact task, and categorize it according to different cg, levels and task types","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":135,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, group, type)(irate(storage_compact_task_file_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_compact_task_file_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg cg{{group}}@{{type}}","metric":"","query":"sum by(le, group, type)(irate(storage_compact_task_file_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_compact_task_file_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task File Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The distribution of the compact task size triggered, including p90 and max. and categorize it according to different cg, levels and task types.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":136,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - cg{{group}}@{{type}}","metric":"","query":"histogram_quantile(0.9, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - cg{{group}}@{{type}}","metric":"","query":"histogram_quantile(1.0, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task Size Distribution","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that are running.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":137,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(storage_compact_task_pending_num{job=~\"$job\",instance=~\"$node\"}) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor_task_split_count - {{job}} @ {{instance}}","metric":"","query":"avg(storage_compact_task_pending_num{job=~\"$job\",instance=~\"$node\"}) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Running Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"compact-task: The total time have been spent on compaction.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":138,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p50 - {{job}}","metric":"","query":"histogram_quantile(0.5, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p90 - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task pmax - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range p90 - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range pmax - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id p90 - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id pmax - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(compute_refill_cache_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compute_apply_version_duration_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(compute_refill_cache_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(compactor_compact_task_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task avg","metric":"","query":"sum by(le)(rate(compactor_compact_task_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(state_store_compact_sst_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range avg","metric":"","query":"sum by(le)(rate(state_store_compact_sst_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"KBs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":139,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job) + sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}}","metric":"","query":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job) + sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","query":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","query":"sum(rate(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_fast_compact_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fast compact - {{job}}","metric":"","query":"sum(rate(compactor_fast_compact_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of bytes that have been written by compaction.Flush refers to the process of compacting Memtables to SSTables at Level 0.Write refers to the process of compacting SSTables at one level to another level.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":140,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","query":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","query":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Bytes(GiB)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Write amplification is the amount of bytes written to the remote storage by compaction for each one byte of flushed SSTable data. Write amplification is by definition higher than 1.0 because we write each piece of data to L0, and then write it again to an SSTable, and then compaction may read this piece of data and write it to a new SSTable, that's another write.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":141,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) / sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"})","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write amplification","metric":"","query":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) / sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"})","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Amplification","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of SSTables that is being compacted at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":142,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"storage_level_compact_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting SSTable Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"num of compact_task","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":143,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_task_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task}}","metric":"","query":"storage_level_compact_task_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":144,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from next level","metric":"","query":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from current level","metric":"","query":"sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} write to next level","metric":"","query":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read/Write by Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":145,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_write_sstn{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} write to next level","metric":"","query":"sum(irate(storage_level_compact_write_sstn{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_read_sstn_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from next level","metric":"","query":"sum(irate(storage_level_compact_read_sstn_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_read_sstn_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from current level","metric":"","query":"sum(irate(storage_level_compact_read_sstn_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read/Write by level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total bytes gotten from sstable_bloom_filter, for observing bloom_filter size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":146,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_meta - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_file - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total bytes gotten from sstable_avg_key_size, for observing sstable_avg_key_size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":147,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_key_size - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_value_size - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Item Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg count gotten from sstable_distinct_epoch_count, for observing sstable_distinct_epoch_count","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":148,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_epoch_count - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Stat","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total time of operations which read from remote storage when enable prefetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":149,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Remote Read Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":150,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_iter_scan_key_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{type}} @ {{instance}} ","metric":"","query":"sum(rate(compactor_iter_scan_key_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Iter keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"bytes of Lsm tree needed to reach balance","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":151,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_pending_bytes{job=~\"$job\",instance=~\"$node\"}) by (instance, group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact pending bytes - {{group}} @ {{instance}} ","metric":"","query":"sum(storage_compact_pending_bytes{job=~\"$job\",instance=~\"$node\"}) by (instance, group)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Compact Pending Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"compression ratio of each level of the lsm tree","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":152,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_level_compression_ratio{job=~\"$job\",instance=~\"$node\"}) by (instance, group, level, algorithm)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lsm compression ratio - cg{{group}} @ L{{level}} - {{algorithm}} {{instance}}","metric":"","query":"sum(storage_compact_level_compression_ratio{job=~\"$job\",instance=~\"$node\"}) by (instance, group, level, algorithm)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Level Compression Ratio","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Compaction","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":12},"height":null,"hideTimeOverride":false,"id":153,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":154,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":155,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":156,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type!~'streaming_upload_write_bytes|streaming_read_read_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count{type!~'streaming_upload_write_bytes|streaming_read_read_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'upload|delete',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count{type=~'upload|delete',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":157,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":158,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Failure Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":159,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(aws_sdk_retry_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(irate(aws_sdk_retry_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(s3_read_request_retry_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(irate(s3_read_request_retry_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Retry Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"There are two types of operations: 1. GET, SELECT, and DELETE, they cost 0.0004 USD per 1000 requests. 2. PUT, COPY, POST, LIST, they cost 0.005 USD per 1000 requests.Reading from S3 across different regions impose extra cost. This metric assumes 0.01 USD per 1GB data transfer. Please checkout AWS's pricing model for more accurate calculation.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":160,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}) * 0.01 / 1000 / 1000 / 1000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"(Cross Region) Data Transfer Cost","metric":"","query":"sum(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}) * 0.01 / 1000 / 1000 / 1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete',job=~\"$job\",instance=~\"$node\"}) * 0.0004 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GET, SELECT, and all other Requests Cost","metric":"","query":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete',job=~\"$job\",instance=~\"$node\"}) * 0.0004 / 1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list',job=~\"$job\",instance=~\"$node\"}) * 0.005 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"PUT, COPY, POST, LIST Requests Cost","metric":"","query":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list',job=~\"$job\",instance=~\"$node\"}) * 0.005 / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Realtime)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric uses the total size of data in S3 at this second to derive the cost of storing data for a whole month. The price is 0.023 USD per GB. Please checkout AWS's pricing model for more accurate calculation.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":161,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance) * 0.023 / 1000 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Monthly Storage Cost","metric":"","query":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance) * 0.023 / 1000 / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Monthly)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Object Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":13},"height":null,"hideTimeOverride":false,"id":162,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":163,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache {{op}} {{extra}} @ {{instance}}","metric":"","query":"sum(rate(foyer_storage_op_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":164,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":165,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","query":"sum(rate(foyer_storage_op_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":166,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(foyer_storage_total_bytes{job=~\"$job\",instance=~\"$node\"}) by (foyer, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} size @ {{instance}}","metric":"","query":"sum(foyer_storage_total_bytes{job=~\"$job\",instance=~\"$node\"}) by (foyer, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":167,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) / (sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) + sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"miss\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache hit ratio @ {{instance}}","metric":"","query":"sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) / (sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) + sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"miss\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Hit Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":168,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(refill_queue_total) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"refill queue length @ {{instance}}","metric":"","query":"sum(refill_queue_total) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Queue Length","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":169,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(refill_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} file cache refill - {{op}} @ {{instance}}","metric":"","query":"sum(rate(refill_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(refill_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} file cache refill - {{op}} @ {{instance}}","metric":"","query":"sum(rate(refill_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":170,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Tiered Cache","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":14},"height":null,"hideTimeOverride":false,"id":171,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":172,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p50 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p99 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time pmax - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lock Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":173,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p50 - {{method}}","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p99 - {{method}}","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time pmax - {{method}}","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Real Process Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":174,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version size","metric":"","query":"storage_version_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":175,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"current version id","metric":"","query":"storage_current_version_id{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"checkpoint version id","metric":"","query":"storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned version id","metric":"","query":"storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_safepoint_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min safepoint version id","metric":"","query":"storage_min_safepoint_version_id{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Id","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":176,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"max committed epoch","metric":"","query":"storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"safe epoch","metric":"","query":"storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned epoch","metric":"","query":"storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":177,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_key_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_value_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_value_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":178,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_materialized_view_stats{metric='materialized_view_total_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{metric}}, mv id - {{table_id}} ","metric":"","query":"storage_materialized_view_stats{metric='materialized_view_total_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":179,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_count',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_key_count',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\nObjects are classified into 3 groups:\n- not referenced by versions: these object are being deleted from object store.\n- referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n- referenced by current version: these objects are in the latest version.\n\nAdditionally, a metric on all objects (including dangling ones) is updated with low-frequency. The metric is updated right before full GC. So subsequent full GC may reduce the actual value significantly, without updating the metric.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":180,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","query":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","query":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","query":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_total_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all objects (including dangling ones)","metric":"","query":"storage_total_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Refer to `Object Total Number` panel for classification of objects.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":181,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","query":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","query":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","query":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_total_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all objects, including dangling ones","metric":"","query":"storage_total_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"total number of hummock version delta log","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":182,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_delta_log_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"delta log total number","metric":"","query":"storage_delta_log_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Delta Log Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"hummock version checkpoint latency","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":183,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(storage_version_checkpoint_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(storage_version_checkpoint_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_avg","metric":"","query":"rate(storage_version_checkpoint_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(storage_version_checkpoint_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Checkpoint Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"When certain per compaction group threshold is exceeded (e.g. number of level 0 sub-level in LSMtree), write op to that compaction group is stopped temporarily. Check log for detail reason of write stop.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":184,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_write_stop_compaction_groups{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compaction_group_{{compaction_group_id}}","metric":"","query":"storage_write_stop_compaction_groups{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Stop Compaction Groups","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"total number of attempts to trigger full GC","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":185,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_full_gc_trigger_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"full_gc_trigger_count","metric":"","query":"storage_full_gc_trigger_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Full GC Trigger Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"the object id watermark used in last full GC","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":186,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_full_gc_last_object_id_watermark{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"full_gc_last_object_id_watermark","metric":"","query":"storage_full_gc_last_object_id_watermark{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Full GC Last Watermark","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":187,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Event Loop Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The times of move_state_table occurs","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":188,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_move_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"move table cg{{group}}","metric":"","query":"sum(storage_move_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}) by (group)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Move State Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of state_tables in each CG","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":189,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"state table cg{{group}}","metric":"","query":"sum(irate(storage_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"State Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of branched_sst in each CG","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":190,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_branched_sst_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"branched sst cg{{group}}","metric":"","query":"sum(irate(storage_branched_sst_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Branched SST Count","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":15},"height":null,"hideTimeOverride":false,"id":191,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total backup job count since the Meta node starts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":192,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"backup_job_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"job count","metric":"","query":"backup_job_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Latency of backup jobs since the Meta node starts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":193,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p50 - {{state}}","metric":"","query":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p99 - {{state}}","metric":"","query":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time pmax - {{state}}","metric":"","query":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Process Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Backup Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":194,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":195,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":196,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Drop latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":197,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetCatalog latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Catalog Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":17},"height":null,"hideTimeOverride":false,"id":198,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":199,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"AddWorkerNode latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":200,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ListAllNodes latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Cluster Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":201,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":202,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"CreateMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":203,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"DropMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":204,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Flush latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Stream Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":19},"height":null,"hideTimeOverride":false,"id":205,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":206,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinVersionBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":207,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinSnapshotBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":208,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ReportCompactionTasks latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":209,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetNewSstIds latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":20},"height":null,"hideTimeOverride":false,"id":210,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":211,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_counts - {{instance}}","metric":"","query":"sum(irate(state_store_report_compaction_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":212,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_version_before_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_avg","metric":"","query":"sum(irate(state_store_unpin_version_before_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"version_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":213,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latencyp90 - {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_avg","metric":"","query":"sum(irate(state_store_pin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_avg","metric":"","query":"sum(irate(state_store_unpin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":214,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_counts - {{instance}}","metric":"","query":"sum(irate(state_store_pin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_counts - {{instance}}","metric":"","query":"sum(irate(state_store_unpin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":215,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_avg","metric":"","query":"sum(irate(state_store_get_new_sst_ids_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":216,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_counts - {{instance}}","metric":"","query":"sum(irate(state_store_get_new_sst_ids_latency_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":217,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_avg","metric":"","query":"sum(irate(state_store_report_compaction_task_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC: Hummock Meta Client","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":21},"height":null,"hideTimeOverride":false,"id":218,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of active sessions","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":219,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Active Sessions","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":220,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per Second (Local Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":221,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per Second (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":222,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","query":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Running Queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":223,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","query":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Rejected queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":224,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","query":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Completed Queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":225,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":226,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency (Local Query Mode)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Frontend","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":22},"height":null,"hideTimeOverride":false,"id":227,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":228,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(lru_runtime_loop_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(lru_runtime_loop_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager loop count per sec","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":229,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_watermark_step{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_watermark_step{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark steps","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"watermark_time is the current lower watermark of cached data. physical_now is the current time of the machine. The diff (physical_now - watermark_time) shows how much data is cached.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":230,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_physical_now_ms{job=~\"$job\",instance=~\"$node\"} - lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_physical_now_ms{job=~\"$job\",instance=~\"$node\"} - lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between watermark_time and now (ms)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":231,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jemalloc_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The allocated memory of jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":232,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_active_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jemalloc_active_bytes{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The active memory of jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":233,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jvm_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jvm_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The allocated memory of jvm","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":234,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jvm_active_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jvm_active_bytes{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The active memory of jvm","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":235,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"} - on() group_right() lru_evicted_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} actor {{actor_id}} desc: {{desc}}","metric":"","query":"lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"} - on() group_right() lru_evicted_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between current watermark and evicted watermark time (ms) for actors","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":23},"height":null,"hideTimeOverride":false,"id":236,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":237,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_type}} @ {{source_id}}","metric":"","query":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":238,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector_type}} @ {{sink_id}}","metric":"","query":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Sink Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Connector Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":239,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":240,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 @ {{connector}} {{sink_id}}","metric":"","query":"histogram_quantile(0.5, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 @ {{connector}} {{sink_id}}","metric":"","query":"histogram_quantile(0.99, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax @ {{connector}} {{sink_id}}","metric":"","query":"histogram_quantile(1.0, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, connector, sink_id)(rate(sink_commit_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(sink_commit_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{connector}} @ {{sink_id}}","metric":"","query":"sum by(le, connector, sink_id)(rate(sink_commit_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(sink_commit_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Commit Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":241,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"latest write epoch @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","query":"log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"latest read epoch @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","query":"log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Read/Write Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":242,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(max(log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Consume lag @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","query":"(max(log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Lag","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":243,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"clamp_min((max(log_store_first_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000, 0)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Consume persistent log lag @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","query":"clamp_min((max(log_store_first_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000, 0)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Consume Persistent Log Lag","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":244,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}}","metric":"","query":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Consume Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":245,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}} @ {{executor_id}} {{instance}}","metric":"","query":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Log Store Consume Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":246,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}}","metric":"","query":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Write Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":247,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}} @ {{executor_id}} {{instance}}","metric":"","query":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Log Store Write Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":248,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_read_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","query":"sum(rate(kv_log_store_storage_read_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Read Storage Row Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":249,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_read_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","query":"sum(rate(kv_log_store_storage_read_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Read Storage Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":250,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_write_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","query":"sum(rate(kv_log_store_storage_write_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Write Storage Row Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":251,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_write_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","query":"sum(rate(kv_log_store_storage_write_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Write Storage Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Sink Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":25},"height":null,"hideTimeOverride":false,"id":252,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Current number of messages in producer queues","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":253,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","query":"rdkafka_top_msg_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Count in Producer Queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Current total size of messages in producer queues","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":254,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_msg_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","query":"rdkafka_top_msg_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Size in Producer Queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of messages transmitted (produced) to Kafka brokers","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":255,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_tx_msgs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","query":"rdkafka_top_tx_msgs{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Produced Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of messages consumed, not including ignored messages (due to offset, etc), from Kafka brokers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":256,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_rx_msgs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","query":"rdkafka_top_rx_msgs{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Received Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages awaiting transmission to broker","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":257,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_outbuf_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","query":"rdkafka_broker_outbuf_msg_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Count Pending to Transmit (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages in-flight to broker awaiting response","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":258,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_waitresp_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","query":"rdkafka_broker_waitresp_msg_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Inflight Message Count (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of transmission errors","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":259,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_tx_errs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","query":"rdkafka_broker_tx_errs{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Error Count When Transmitting (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of receive errors","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":260,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rx_errs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","query":"rdkafka_broker_rx_errs{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Error Count When Receiving (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of requests timed out","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":261,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_req_timeouts{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","query":"rdkafka_broker_req_timeouts{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Timeout Request Count (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Broker latency / round-trip time in milli seconds","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":262,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_avg{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_avg{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p75{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_p75{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p90{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_p90{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_p99{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"RTT (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Broker throttling time in milliseconds","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":263,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_avg{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_avg{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p75{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_p75{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p90{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_p90{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_p99{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throttle Time (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Age of metadata from broker for this topic (milliseconds)","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":264,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_metadata_age{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}","metric":"","query":"rdkafka_topic_metadata_age{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Topic Metadata_age Age","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Batch sizes in bytes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":265,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_avg{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_avg{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p75{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_p75{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p90{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_p90{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_p99{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p99_99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_p99_99{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_out_of_range{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_out_of_range{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Batch message counts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":null,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_avg{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_avg{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p75{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_p75{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p90{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_p90{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_p99{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p99_99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_p99_99{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_out_of_range{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_out_of_range{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Topic Batch Messages","transformations":[],"transparent":false,"type":"timeseries"}],"timeFrom":null,"timeShift":null,"title":"Topic Batch Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages ready to be produced in transmit queue","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":266,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_xmit_msgq_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","query":"rdkafka_topic_partition_xmit_msgq_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message to be Transmitted","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of pre-fetched messages in fetch queue","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":267,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_fetchq_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","query":"rdkafka_topic_partition_fetchq_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message in pre fetch queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Next offset to fetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":268,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_next_offset{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","query":"rdkafka_topic_partition_next_offset{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Next offset to fetch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Last committed offset","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":269,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_committed_offset{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","query":"rdkafka_topic_partition_committed_offset{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Committed Offset","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Kafka Native Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":270,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":271,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} read @ {{instance}}","metric":"","query":"sum(rate(connection_read_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} write @ {{instance}}","metric":"","query":"sum(rate(connection_write_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Network throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":272,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} read @ {{instance}}","metric":"","query":"sum(rate(connection_read_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} write @ {{instance}}","metric":"","query":"sum(rate(connection_write_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"S3 throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":273,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} read @ {{instance}}","metric":"","query":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} write @ {{instance}}","metric":"","query":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total read @ {{instance}}","metric":"","query":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total write @ {{instance}}","metric":"","query":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"gRPC throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":274,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_io_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","query":"sum(irate(connection_io_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_io_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} grpc {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","query":"sum(rate(connection_io_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_io_err_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","query":"sum(rate(connection_io_err_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"IO error rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":275,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(connection_count{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","query":"sum(connection_count{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(connection_count{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","query":"sum(connection_count{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}) by (job, instance, connection_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Existing connection count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":276,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_create_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","query":"sum(irate(connection_create_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_create_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","query":"sum(irate(connection_create_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create new connection rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":277,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","query":"sum(irate(connection_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","query":"sum(irate(connection_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create new connection err rate","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Network connection","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":27},"height":null,"hideTimeOverride":false,"id":278,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"iceberg file appender write qps","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":279,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"iceberg_file_appender_write_qps{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} @ {{sink_id}}","metric":"","query":"iceberg_file_appender_write_qps{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Qps Of Iceberg File Appender","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":280,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 @ {{sink_id}}","metric":"","query":"histogram_quantile(0.5, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 @ {{sink_id}}","metric":"","query":"histogram_quantile(0.99, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax @ {{sink_id}}","metric":"","query":"histogram_quantile(1.0, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, sink_id)(rate(iceberg_file_appender_write_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(iceberg_file_appender_write_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg @ {{sink_id}}","metric":"","query":"sum by(le, sink_id)(rate(iceberg_file_appender_write_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(iceberg_file_appender_write_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write latency Of Iceberg File Appender","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Iceberg Sink Metrics","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, instance)","description":"Reporting instance of the metric","hide":0,"includeAll":true,"label":"Node","multi":true,"name":"node","options":[],"query":{"query":"label_values(process_cpu_seconds_total, instance)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, job)","description":"Reporting job of the metric","hide":0,"includeAll":true,"label":"Job","multi":true,"name":"job","options":[],"query":{"query":"label_values(process_cpu_seconds_total, job)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(table_info, table_id)","description":"Reporting table id of the metric","hide":0,"includeAll":true,"label":"Table","multi":true,"name":"table","options":[],"query":{"query":"label_values(table_info, table_id)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"}]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dev_dashboard","uid":"Ecy3uV1nz","version":0} +{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dev Dashboard","editable":true,"gnetId":null,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from actor id to fragment id","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Id Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from materialized view table id to it's internal table ids","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"table_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Actor/Table Id Info","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of each type of RisingWave components alive.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":5,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The memory usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":6,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The CPU usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":7,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cpu usage (total) - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cpu usage (avg per core) - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"RW cluster can configure multiple meta nodes to achieve high availability. One is the leader and the rest are the followers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":8,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(meta_num{job=~\"$job\",instance=~\"$node\"}) by (worker_addr,role)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_addr}} @ {{role}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Meta Cluster","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Cluster Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":2},"height":null,"hideTimeOverride":false,"id":9,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The rate of successful recovery attempts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":10,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Recovery Successful Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of failed reocovery attempts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":11,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Failed recovery attempts","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Time spent in a successful recovery attempt","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":12,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency pmax - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by (le) (rate(recovery_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by (le) (rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Recovery latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Recovery","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":3},"height":null,"hideTimeOverride":false,"id":13,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":14,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Each query is executed in parallel with a user-defined parallelism. This figure shows the throughput of each parallelism. The throughput of all the parallelism added up is equal to Source Throughput(rows).","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":15,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(partition_input_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":16,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Each query is executed in parallel with a user-defined parallelism. This figure shows the throughput of each parallelism. The throughput of all the parallelism added up is equal to Source Throughput(MB/s).","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":17,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"RisingWave ingests barriers periodically to trigger computation and checkpoints. The frequency of barrier can be set by barrier_interval_ms. This metric shows how many rows are ingested between two consecutive barriers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":18,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_rows_per_barrier_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) per barrier","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Monitor each source upstream, 0 means the upstream is not normal, 1 means the source is ready.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":19,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"source_status_is_up{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source_id={{source_id}}, source_name={{source_name}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Upstream Status","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Source Split Change Events frequency by source_id and actor_id","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":20,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_split_change_event_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Split Change Events frequency(events/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Kafka Consumer Lag Size by source_id, partition and actor_id","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":21,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"high_watermark{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}} partition={{partition}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"latest_message_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}} partition={{partition}} actor_id={{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kafka Consumer Lag Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":22,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":23,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id, actor_id) * on(actor_id) group_left(sink_name) sink_info{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}} - actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s) per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":24,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":25,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (actor_id, table_id) * on(actor_id, table_id) group_left(table_name) table_info{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}} - actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s) per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been read from the backfill snapshot","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":26,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_snapshot_read_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Snapshot Read Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been output from the backfill upstream","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":27,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_upstream_output_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Upstream Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of barriers that have been ingested but not completely processed. This metric reflects the current level of congestion within the system.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":28,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all_barrier","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"in_flight_barrier_nums{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"in_flight_barrier","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The duration between the time point when the scheduled barrier needs to be sent and the time point when the barrier gets actually sent to all the compute nodes. Developers can thus detect any internal congestion.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":29,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_send_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Send Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The time that the data between two consecutive barriers gets fully processed, i.e. the computation results are made durable into materialized views or sink to external systems. This metric shows to users the freshness of materialized views.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":30,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":31,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier In-Flight Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":32,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p999 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_pmax - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_avg - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Sync Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":33,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_wait_commit_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Wait Commit Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of actors that have processed the earliest in-flight barriers per second. This metric helps users to detect potential congestion or stuck in the system.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":34,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_barrier_manager_progress{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Earliest In-Flight Barrier Progress","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":4},"height":null,"hideTimeOverride":false,"id":35,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"We first record the total blocking duration(ns) of output buffer of each actor. It shows how much time it takes an actor to process a message, i.e. a barrier, a watermark or rows of data, on average. Then we divide this duration by 1 second and show it as a percentage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":36,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}->{{downstream_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Blocking Time Ratio (Backpressure)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":37,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_input_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, upstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}<-{{upstream_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":38,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":39,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The operator-level memory usage statistics collected by each LRU cache","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":40,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (table_id, desc)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} desc: {{desc}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_memory_usage{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} actor {{actor_id}} desc: {{desc}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Memory Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Memory usage aggregated by materialized views","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":41,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Memory Usage of Materialized Views","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":42,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"temporal join cache miss, table_id {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"temporal join cache miss, table_id {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Temporal Join Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":43,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache hit count - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total cached count - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache hit count - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total cached count - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialize Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":44,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache lookup count - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss count - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache lookup count - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss count - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Over Window Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":45,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"join executor cache miss ratio - - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n appendonly cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream lookup cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream temporal join cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialize executor cache miss ratio - table {{table_id}} fragment {{fragment_id}} {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_over_window_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_over_window_cache_lookup_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Over window cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Miss Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":46,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, fragment_id, wait_side, job)(rate(stream_join_barrier_align_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le,fragment_id,wait_side,job) (rate(stream_join_barrier_align_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Barrier Align","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":47,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":48,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}} {{side}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Match Duration Per Second","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Multiple rows with distinct primary keys may have the same join key. This metric counts the number of join keys in the executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":49,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (fragment_id, side)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}} {{side}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of matched rows on the opposite side","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":50,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, actor_id, table_id) (rate(stream_join_matched_join_keys_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, fragment_id, table_id) (rate(stream_join_matched_join_keys_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Matched Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":51,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss - table {{table_id}} actor {{actor_id}}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each StreamChunk","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":52,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg cached keys count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg distinct cached keys count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg cached keys count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg distinct cached keys count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of dirty (unflushed) groups in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":53,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Dirty Groups Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The total heap size of dirty (unflushed) groups in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":54,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups heap size | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups heap size | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Dirty Groups Heap Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in each top_n executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":55,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n appendonly cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n cached count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n appendonly cached count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"TopN Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in temporal join executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":56,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal Join cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal Join cached count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Temporal Join Cache Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in lookup executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":57,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lookup cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lookup cached count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lookup Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in over window executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":58,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"over window cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"over window cached count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Over Window Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"When enabled, this metric shows the input throughput of each executor.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":59,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_identity, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_identity}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_identity}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The actor-level memory usage statistics reported by TaskLocalAlloc. (Disabled by default)","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":60,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(actor_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_memory_usage{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Memory Usage (TaskLocalAlloc)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":5},"height":null,"hideTimeOverride":false,"id":61,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":62,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_actor_execution_time{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Execution Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":63,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":64,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":8},"height":null,"hideTimeOverride":false,"id":65,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":8},"height":null,"hideTimeOverride":false,"id":66,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":67,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":16},"height":null,"hideTimeOverride":false,"id":68,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":16},"height":null,"hideTimeOverride":false,"id":69,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":70,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":24},"height":null,"hideTimeOverride":false,"id":71,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":24},"height":null,"hideTimeOverride":false,"id":72,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":73,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":32},"height":null,"hideTimeOverride":false,"id":74,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":32},"height":null,"hideTimeOverride":false,"id":75,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":76,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":40},"height":null,"hideTimeOverride":false,"id":77,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Avg Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors (Tokio)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":6},"height":null,"hideTimeOverride":false,"id":78,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":79,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Send Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":80,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Recv Throughput","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Exchange","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":7},"height":null,"hideTimeOverride":false,"id":81,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":82,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compute Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":83,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":84,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_reader_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: actor_id={{actor_id}}, source_id={{source_id}})","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_reader_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: actor_id={{actor_id}}, source_id={{source_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Reader Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":85,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_sink_error{job=~\"$job\",instance=~\"$node\"}) by (connector_name, executor_id, error_msg)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{connector_name}}: {{error_msg}} ({{executor_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink by Connector","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"User Streaming Errors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":86,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":87,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Exchange Recv Row Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":88,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mpp Task Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"All memory usage of batch executors in bytes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":89,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_total_mem{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mem Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":90,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_heartbeat_worker_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Heartbeat Worker Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the memory usage of mem_table.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":91,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table size total - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table size - table id {{table_id}} instance id {{instance_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Mem Table Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the item counts in mem_table.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":92,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table counts total - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table count - table id {{table_id}} instance id {{instance_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Mem Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":93,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Row SeqScan Next Duration","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":94,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":95,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{table_id}} @ {{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total_meta_miss_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Hummock has three parts of memory usage: 1. Meta Cache 2. Block CacheThis metric shows the real memory usage of each of these three caches.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":96,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"data cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":97,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"block cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Miss Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":98,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_scan_key_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type, table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{table_id}} @ {{type}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Iter keys flow","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":99,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p50 - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p99 - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts pmax - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Merged SSTs","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of the latency of Get operations that have been issued to the state store.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":100,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of the time spent on iterator initialization.Histogram of the time spent on iterator scanning.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":101,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_init_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_init_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_scan_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_scan_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":102,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive count - {{table_id}} - {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter positive count - {{table_id}} - {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter check count- {{table_id}} - {{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Positive / Total","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":103,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter positive rate - {{table_id}} - {{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter Positive Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"False-Positive / Total","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":104,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(((sum(rate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter false positive rate - {{table_id}} - {{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter False-Positive Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":105,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_iter_slow_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Slow Fetch Meta Unhits","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":106,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_shared_buffer_hit_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer hit - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_in_process_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":107,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":108,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":109,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read p50 - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read p99 - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read pmax - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Read Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":110,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Count - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size of a single key-value pair when reading by operation Get.Operation Get gets a single key-value pair with respect to a caller-specified key. If the key does not exist in the storage, the size of key is counted into this metric and the size of value is 0.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":111,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size of all the key-value paris when reading by operation Iter.Operation Iter scans a range of key-value pairs.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":112,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":113,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":114,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_iter_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Unhits","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock (Read)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":10},"height":null,"hideTimeOverride":false,"id":115,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the real memory usage of uploader.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":116,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading memory - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_uploader_uploading_task_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading task size - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader Memory Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of time spent on compacting shared buffer to remote storage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":117,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Build and Sync Sstable Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":118,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.5, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write p50 - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.99, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write p99 - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write pmax - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Write Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":119,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_merge_imm_task_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"merge imm tasks - {{table_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_spill_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Uploader spill tasks - {{uploader_stage}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader - Tasks Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":120,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_merge_imm_memory_sz{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Merging tasks memory size - {{table_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_spill_task_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Uploading tasks size - {{uploader_stage}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader - Task Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":121,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write batch - {{table_id}} @ {{job}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"l0 - {{job}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":122,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":123,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_write_batch_tuple_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write_batch_kv_pair_count - {{table_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Item Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":124,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_size_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id) / sum(rate(state_store_write_batch_size_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_shared_buffer_to_sstable_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance) / sum(rate(compactor_shared_buffer_to_sstable_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sync - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the statistics of mem_table size on flush. By default only max (p100) is shown.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":125,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, table_id, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_write_batch_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, table_id, job, instance) (rate(state_store_write_batch_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Batch Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":126,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_mem_table_spill_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem table spill table id - {{table_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Mem Table Spill Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":127,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Sync Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock (Write)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":11},"height":null,"hideTimeOverride":false,"id":128,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of SSTables at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":129,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SSTable Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size(KB) of SSTables at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":130,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SSTable Size(KB)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The of bytes that have been written by commit epoch per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":131,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_commit_write_throughput{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{table_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Commit Flush Bytes by Table","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have completed or failed","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":132,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency{result!='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Failure Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have completed or failed","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":133,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency{result='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Success Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have been skipped.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":134,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_skip_compact_frequency{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (level, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{level}}-{{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Skip Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg l0 select_level_count of the compact task, and categorize it according to different cg, levels and task types","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":135,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, group, type)(irate(storage_l0_compact_level_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_l0_compact_level_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg cg{{group}}@{{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task L0 Select Level Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg file count of the compact task, and categorize it according to different cg, levels and task types","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":136,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, group, type)(irate(storage_compact_task_file_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_compact_task_file_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg cg{{group}}@{{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task File Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The distribution of the compact task size triggered, including p90 and max. and categorize it according to different cg, levels and task types.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":137,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - cg{{group}}@{{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - cg{{group}}@{{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task Size Distribution","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that are running.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":138,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(storage_compact_task_pending_num{job=~\"$job\",instance=~\"$node\"}) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor_task_split_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Running Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"compact-task: The total time have been spent on compaction.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":139,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p50 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p90 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task pmax - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range p90 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range pmax - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id p90 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id pmax - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(compute_refill_cache_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compute_apply_version_duration_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(compactor_compact_task_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(state_store_compact_sst_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"KBs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":140,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job) + sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_fast_compact_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fast compact - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of bytes that have been written by compaction.Flush refers to the process of compacting Memtables to SSTables at Level 0.Write refers to the process of compacting SSTables at one level to another level.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":141,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Bytes(GiB)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Write amplification is the amount of bytes written to the remote storage by compaction for each one byte of flushed SSTable data. Write amplification is by definition higher than 1.0 because we write each piece of data to L0, and then write it again to an SSTable, and then compaction may read this piece of data and write it to a new SSTable, that's another write.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":142,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) / sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"})","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write amplification","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Amplification","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of SSTables that is being compacted at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":143,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting SSTable Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"num of compact_task","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":144,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_task_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":145,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from next level","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from current level","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} write to next level","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read/Write by Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":146,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_write_sstn{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} write to next level","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_read_sstn_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from next level","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_read_sstn_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from current level","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read/Write by level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total bytes gotten from sstable_bloom_filter, for observing bloom_filter size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":147,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_meta - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_file - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total bytes gotten from sstable_avg_key_size, for observing sstable_avg_key_size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":148,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_key_size - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_value_size - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Item Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg count gotten from sstable_distinct_epoch_count, for observing sstable_distinct_epoch_count","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":149,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_epoch_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Stat","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total time of operations which read from remote storage when enable prefetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":150,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Remote Read Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":151,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_iter_scan_key_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{type}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Iter keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"bytes of Lsm tree needed to reach balance","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":152,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_pending_bytes{job=~\"$job\",instance=~\"$node\"}) by (instance, group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact pending bytes - {{group}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Compact Pending Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"compression ratio of each level of the lsm tree","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":153,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_level_compression_ratio{job=~\"$job\",instance=~\"$node\"}) by (instance, group, level, algorithm)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lsm compression ratio - cg{{group}} @ L{{level}} - {{algorithm}} {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Level Compression Ratio","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Compaction","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":12},"height":null,"hideTimeOverride":false,"id":154,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":155,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":156,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":157,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type!~'streaming_upload_write_bytes|streaming_read_read_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'upload|delete',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-write - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-read - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":158,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":159,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Failure Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":160,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(aws_sdk_retry_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(s3_read_request_retry_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Retry Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"There are two types of operations: 1. GET, SELECT, and DELETE, they cost 0.0004 USD per 1000 requests. 2. PUT, COPY, POST, LIST, they cost 0.005 USD per 1000 requests.Reading from S3 across different regions impose extra cost. This metric assumes 0.01 USD per 1GB data transfer. Please checkout AWS's pricing model for more accurate calculation.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":161,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}) * 0.01 / 1000 / 1000 / 1000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"(Cross Region) Data Transfer Cost","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete',job=~\"$job\",instance=~\"$node\"}) * 0.0004 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GET, SELECT, and all other Requests Cost","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list',job=~\"$job\",instance=~\"$node\"}) * 0.005 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"PUT, COPY, POST, LIST Requests Cost","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Realtime)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric uses the total size of data in S3 at this second to derive the cost of storing data for a whole month. The price is 0.023 USD per GB. Please checkout AWS's pricing model for more accurate calculation.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":162,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance) * 0.023 / 1000 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Monthly Storage Cost","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Monthly)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Object Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":13},"height":null,"hideTimeOverride":false,"id":163,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":164,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache {{op}} {{extra}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":165,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":166,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":167,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(foyer_storage_total_bytes{job=~\"$job\",instance=~\"$node\"}) by (foyer, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} size @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":168,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) / (sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) + sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"miss\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache hit ratio @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Hit Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":169,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(refill_queue_total) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"refill queue length @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Queue Length","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":170,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(refill_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} file cache refill - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(refill_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} file cache refill - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":171,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Tiered Cache","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":14},"height":null,"hideTimeOverride":false,"id":172,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":173,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p50 - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p99 - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time pmax - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lock Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":174,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p50 - {{method}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p99 - {{method}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time pmax - {{method}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Real Process Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":175,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version size","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":176,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"current version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"checkpoint version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_safepoint_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min safepoint version id","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Id","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":177,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"max committed epoch","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"safe epoch","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned epoch","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":178,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_value_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":179,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_materialized_view_stats{metric='materialized_view_total_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{metric}}, mv id - {{table_id}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":180,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_count',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\nObjects are classified into 3 groups:\n- not referenced by versions: these object are being deleted from object store.\n- referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n- referenced by current version: these objects are in the latest version.\n\nAdditionally, a metric on all objects (including dangling ones) is updated with low-frequency. The metric is updated right before full GC. So subsequent full GC may reduce the actual value significantly, without updating the metric.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":181,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_total_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all objects (including dangling ones)","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Refer to `Object Total Number` panel for classification of objects.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":182,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_total_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all objects, including dangling ones","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"total number of hummock version delta log","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":183,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_delta_log_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"delta log total number","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Delta Log Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"hummock version checkpoint latency","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":184,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(storage_version_checkpoint_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(storage_version_checkpoint_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Checkpoint Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"When certain per compaction group threshold is exceeded (e.g. number of level 0 sub-level in LSMtree), write op to that compaction group is stopped temporarily. Check log for detail reason of write stop.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":185,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_write_stop_compaction_groups{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compaction_group_{{compaction_group_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Stop Compaction Groups","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"total number of attempts to trigger full GC","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":186,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_full_gc_trigger_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"full_gc_trigger_count","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Full GC Trigger Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"the object id watermark used in last full GC","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":187,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_full_gc_last_object_id_watermark{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"full_gc_last_object_id_watermark","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Full GC Last Watermark","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":188,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Event Loop Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The times of move_state_table occurs","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":189,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_move_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"move table cg{{group}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Move State Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of state_tables in each CG","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":190,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"state table cg{{group}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"State Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of branched_sst in each CG","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":191,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_branched_sst_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"branched sst cg{{group}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Branched SST Count","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":15},"height":null,"hideTimeOverride":false,"id":192,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total backup job count since the Meta node starts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":193,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"backup_job_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"job count","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Latency of backup jobs since the Meta node starts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":194,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p50 - {{state}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p99 - {{state}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time pmax - {{state}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Process Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Backup Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":195,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":196,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":197,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Drop latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":198,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetCatalog latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Catalog Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":17},"height":null,"hideTimeOverride":false,"id":199,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":200,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"AddWorkerNode latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":201,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ListAllNodes latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Cluster Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":202,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":203,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"CreateMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":204,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"DropMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":205,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Flush latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Stream Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":19},"height":null,"hideTimeOverride":false,"id":206,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":207,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinVersionBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":208,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinSnapshotBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":209,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ReportCompactionTasks latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":210,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetNewSstIds latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":20},"height":null,"hideTimeOverride":false,"id":211,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":212,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_counts - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":213,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_version_before_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"version_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":214,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latencyp90 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":215,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_counts - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_counts - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":216,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":217,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_counts - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":218,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC: Hummock Meta Client","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":21},"height":null,"hideTimeOverride":false,"id":219,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of active sessions","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":220,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Active Sessions","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":221,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per Second (Local Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":222,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per Second (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":223,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Running Queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":224,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Rejected queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":225,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Completed Queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":226,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":227,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency (Local Query Mode)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Frontend","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":22},"height":null,"hideTimeOverride":false,"id":228,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":229,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(lru_runtime_loop_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager loop count per sec","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":230,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_watermark_step{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark steps","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"watermark_time is the current lower watermark of cached data. physical_now is the current time of the machine. The diff (physical_now - watermark_time) shows how much data is cached.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":231,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_physical_now_ms{job=~\"$job\",instance=~\"$node\"} - lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between watermark_time and now (ms)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":232,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The allocated memory of jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":233,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_active_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The active memory of jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":234,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jvm_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The allocated memory of jvm","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":235,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jvm_active_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The active memory of jvm","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":236,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"} - on() group_right() lru_evicted_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} actor {{actor_id}} desc: {{desc}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between current watermark and evicted watermark time (ms) for actors","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":23},"height":null,"hideTimeOverride":false,"id":237,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":238,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_type}} @ {{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":239,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector_type}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Sink Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Connector Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":240,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":241,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 @ {{connector}} {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 @ {{connector}} {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax @ {{connector}} {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, connector, sink_id)(rate(sink_commit_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(sink_commit_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{connector}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Commit Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":242,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"latest write epoch @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"latest read epoch @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Read/Write Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":243,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(max(log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Consume lag @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Lag","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":244,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"clamp_min((max(log_store_first_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000, 0)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Consume persistent log lag @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Consume Persistent Log Lag","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":245,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Consume Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":246,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}} @ {{executor_id}} {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Log Store Consume Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":247,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Write Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":248,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}} @ {{executor_id}} {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Log Store Write Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":249,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_read_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Read Storage Row Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":250,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_read_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Read Storage Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":251,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_write_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Write Storage Row Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":252,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_write_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Write Storage Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Sink Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":25},"height":null,"hideTimeOverride":false,"id":253,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Current number of messages in producer queues","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":254,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Count in Producer Queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Current total size of messages in producer queues","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":255,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_msg_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Size in Producer Queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of messages transmitted (produced) to Kafka brokers","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":256,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_tx_msgs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Produced Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of messages consumed, not including ignored messages (due to offset, etc), from Kafka brokers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":257,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_rx_msgs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Received Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages awaiting transmission to broker","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":258,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_outbuf_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Count Pending to Transmit (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages in-flight to broker awaiting response","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":259,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_waitresp_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Inflight Message Count (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of transmission errors","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":260,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_tx_errs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Error Count When Transmitting (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of receive errors","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":261,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rx_errs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Error Count When Receiving (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of requests timed out","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":262,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_req_timeouts{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Timeout Request Count (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Broker latency / round-trip time in milli seconds","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":263,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_avg{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p75{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p90{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"RTT (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Broker throttling time in milliseconds","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":264,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_avg{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p75{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p90{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throttle Time (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Age of metadata from broker for this topic (milliseconds)","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":265,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_metadata_age{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Topic Metadata_age Age","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Batch sizes in bytes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":266,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_avg{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p75{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p90{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p99_99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_out_of_range{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Batch message counts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":null,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_avg{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p75{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p90{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p99_99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_out_of_range{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Topic Batch Messages","transformations":[],"transparent":false,"type":"timeseries"}],"timeFrom":null,"timeShift":null,"title":"Topic Batch Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages ready to be produced in transmit queue","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":267,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_xmit_msgq_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message to be Transmitted","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of pre-fetched messages in fetch queue","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":268,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_fetchq_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message in pre fetch queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Next offset to fetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":269,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_next_offset{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Next offset to fetch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Last committed offset","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":270,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_committed_offset{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Committed Offset","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Kafka Native Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":271,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":272,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} read @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} write @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Network throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":273,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} read @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} write @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"S3 throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":274,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} read @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} write @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total read @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total write @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"gRPC throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":275,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_io_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_io_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} grpc {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_io_err_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"IO error rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":276,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(connection_count{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(connection_count{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Existing connection count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":277,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_create_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_create_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create new connection rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":278,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create new connection err rate","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Network connection","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":27},"height":null,"hideTimeOverride":false,"id":279,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"iceberg file appender write qps","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":280,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"iceberg_file_appender_write_qps{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Qps Of Iceberg File Appender","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":281,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 @ {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 @ {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax @ {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, sink_id)(rate(iceberg_file_appender_write_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(iceberg_file_appender_write_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write latency Of Iceberg File Appender","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Iceberg Sink Metrics","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, instance)","description":"Reporting instance of the metric","hide":0,"includeAll":true,"label":"Node","multi":true,"name":"node","options":[],"query":{"query":"label_values(process_cpu_seconds_total, instance)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, job)","description":"Reporting job of the metric","hide":0,"includeAll":true,"label":"Job","multi":true,"name":"job","options":[],"query":{"query":"label_values(process_cpu_seconds_total, job)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(table_info, table_id)","description":"Reporting table id of the metric","hide":0,"includeAll":true,"label":"Table","multi":true,"name":"table","options":[],"query":{"query":"label_values(table_info, table_id)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"}]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dev_dashboard","uid":"Ecy3uV1nz","version":0} diff --git a/docker/dashboards/risingwave-user-dashboard.json b/docker/dashboards/risingwave-user-dashboard.json index 63e3ed095ac69..41445ec1e30fb 100644 --- a/docker/dashboards/risingwave-user-dashboard.json +++ b/docker/dashboards/risingwave-user-dashboard.json @@ -1 +1 @@ -{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dashboard","editable":true,"gnetId":null,"graphTooltip":0,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Actor/Table Id Info","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from actor id to fragment id","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"actor_info{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Id Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from materialized view table id to it's internal table ids","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":12,"y":1},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id, table_name, table_type)","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id, table_name, table_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Overview","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":10},"height":null,"hideTimeOverride":false,"id":5,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (source_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{source_name}}","metric":"","query":"sum(rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (source_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregated Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":10},"height":null,"hideTimeOverride":false,"id":6,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source_id {{source_id}}","metric":"","query":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregated Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":7,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}}","metric":"","query":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":18},"height":null,"hideTimeOverride":false,"id":8,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}}","metric":"","query":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The time that the data between two consecutive barriers gets fully processed, i.e. the computation results are made durable into materialized views or sink to external systems. This metric shows to users the freshness of materialized views.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":9,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","query":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Alerts in the system group by type:\n - Too Many Barriers: there are too many uncommitted barriers generated. This means the streaming graph is stuck or under heavy load. Check 'Barrier Latency' panel.\n - Recovery Triggered: cluster recovery is triggered. Check 'Errors by Type' / 'Node Count' panels.\n - Lagging Version: the checkpointed or pinned version id is lagging behind the current version id. Check 'Hummock Manager' section in dev dashboard.\n - Lagging Epoch: the pinned or safe epoch is lagging behind the current max committed epoch. Check 'Hummock Manager' section in dev dashboard.\n - Lagging Compaction: there are too many files in L0. This can be caused by compactor failure or lag of compactor resource. Check 'Compaction' section in dev dashboard.\n - Lagging Vacuum: there are too many stale files waiting to be cleaned. This can be caused by compactor failure or lag of compactor resource. Check 'Compaction' section in dev dashboard.\n - Abnormal Meta Cache Memory: the meta cache memory usage is too large, exceeding the expected 10 percent.\n - Abnormal Block Cache Memory: the block cache memory usage is too large, exceeding the expected 10 percent.\n - Abnormal Uploading Memory Usage: uploading memory is more than 70 percent of the expected, and is about to spill.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":26},"height":null,"hideTimeOverride":false,"id":10,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"} >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Too Many Barriers","metric":"","query":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"} >= bool 200","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) > bool 0 + sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) > bool 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Recovery Triggered","metric":"","query":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) > bool 0 + sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) > bool 0","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100) + ((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Version","metric":"","query":"((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100) + ((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"} == 0) + ((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_safe_epoch{job=~\"$job\",instance=~\"$node\"} == 0)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Epoch","metric":"","query":"((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"} == 0) + ((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_safe_epoch{job=~\"$job\",instance=~\"$node\"} == 0)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(label_replace(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}, 'L0', 'L0', 'level_index', '.*_L0') unless storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (L0) >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Compaction","metric":"","query":"sum(label_replace(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}, 'L0', 'L0', 'level_index', '.*_L0') unless storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (L0) >= bool 200","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"} >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Vacuum","metric":"","query":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"} >= bool 200","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_meta_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Meta Cache Memory","metric":"","query":"state_store_meta_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_block_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Block Cache Memory","metric":"","query":"state_store_block_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_uploading_memory_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 0.7","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Uploading Memory Usage","metric":"","query":"state_store_uploading_memory_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 0.7","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Alerts","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Errors in the system group by type","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":34},"height":null,"hideTimeOverride":false,"id":11,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compute error {{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","query":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"parse error {{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","query":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"source_status_is_up{job=~\"$job\",instance=~\"$node\"} == 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source error: source_id={{source_id}}, source_name={{source_name}} @ {{instance}}","metric":"","query":"source_status_is_up{job=~\"$job\",instance=~\"$node\"} == 0","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote storage error {{type}}: {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Errors","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":34},"height":null,"hideTimeOverride":false,"id":12,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Local mode","metric":"","query":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distributed mode","metric":"","query":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Query QPS","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of each type of RisingWave components alive.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":42},"height":null,"hideTimeOverride":false,"id":13,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","query":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of active sessions in frontend nodes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":42},"height":null,"hideTimeOverride":false,"id":14,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Active Sessions","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":50},"height":null,"hideTimeOverride":false,"id":15,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The CPU usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":16,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","query":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of CPU cores per RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":17,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","query":"avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU Core Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"CPU","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":51},"height":null,"hideTimeOverride":false,"id":18,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The memory usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":19,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":20,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage @ {{instance}}","metric":"","query":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Memory Usage (Total)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":21,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(actor_memory_usage[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"streaming actor - {{actor_id}}","metric":"","query":"rate(actor_memory_usage[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage meta cache - {{job}} @ {{instance}}","metric":"","query":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage block cache - {{job}} @ {{instance}}","metric":"","query":"sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage write buffer - {{job}} @ {{instance}}","metric":"","query":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized_view {{materialized_view_id}}","metric":"","query":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info) by (materialized_view_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Memory Usage (Detailed)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Executor cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":22,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Join - cache miss - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Join - total lookups - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n appendonly - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n appendonly - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lookup executor - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lookup executor - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal join - cache miss - table_id {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal join - total lookups - table_id {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Materialize - cache hit count - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","query":"rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Materialize - total cache count - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","query":"rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":23,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"join executor cache miss ratio - - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","query":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n appendonly cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream lookup cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream temporal join cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialize executor cache miss ratio - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","query":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Miss Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":24,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"memory cache - {{table_id}} @ {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_sst_store_block_request_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total_meta_miss_count - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage bloom filter statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":25,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_check_bloom_filter_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter total - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_check_bloom_filter_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_positive_but_non_exist_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_positive_but_non_exist_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Bloom Filer","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage file cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":26,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","query":"sum(rate(file_cache_miss{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage File Cache","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":52},"height":null,"hideTimeOverride":false,"id":27,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Send/Recv throughput per node for streaming exchange","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":28,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Send @ {{instance}}","metric":"","query":"sum(rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Recv @ {{instance}}","metric":"","query":"sum(rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Streming Remote Exchange (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The remote storage read/write throughput per node","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":29,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{instance}}","metric":"","query":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{instance}}","metric":"","query":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Remote I/O (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":30,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","query":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Exchange Recv (Rows/s)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Network","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":53},"height":null,"hideTimeOverride":false,"id":31,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\n Objects are classified into 3 groups:\n - not referenced by versions: these object are being deleted from object store.\n - referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n - referenced by current version: these objects are in the latest version.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":32,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","query":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","query":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","query":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The storage size of each materialized view","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":33,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_materialized_view_stats{metric='materialized_view_total_size',job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{metric}}, mv id - {{table_id}} ","metric":"","query":"storage_materialized_view_stats{metric='materialized_view_total_size',job=~\"$job\",instance=~\"$node\"}/1024","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\n Objects are classified into 3 groups:\n - not referenced by versions: these object are being deleted from object store.\n - referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n - referenced by current version: these objects are in the latest version.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":34,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","query":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","query":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","query":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of bytes that have been written by compaction.Flush refers to the process of compacting Memtables to SSTables at Level 0.Compaction refers to the process of compacting SSTables at one level to another level.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":35,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Compaction - {{job}}","metric":"","query":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush - {{job}}","metric":"","query":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The remote storage read/write throughput","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":36,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}}","metric":"","query":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","query":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Remote I/O (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Size statistics for checkpoint","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":37,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}}","metric":"","query":"sum by(le, job) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":54},"height":null,"hideTimeOverride":false,"id":38,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":39,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":40,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","query":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized executor actor per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":41,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_executor_row_count{executor_identity=~\".*MaterializeExecutor.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) * on(actor_id) group_left(materialized_view_id, table_name) (group(table_info{table_type=~\"MATERIALIZED_VIEW\",job=~\"$job\",instance=~\"$node\"}) by (actor_id, materialized_view_id, table_name))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized view {{table_name}} table_id {{materialized_view_id}}","metric":"","query":"sum(rate(stream_executor_row_count{executor_identity=~\".*MaterializeExecutor.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) * on(actor_id) group_left(materialized_view_id, table_name) (group(table_info{table_type=~\"MATERIALIZED_VIEW\",job=~\"$job\",instance=~\"$node\"}) by (actor_id, materialized_view_id, table_name))) by (materialized_view_id, table_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been read from the backfill operator used by MV on MV","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":42,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_snapshot_read_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Read Snapshot - table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_backfill_snapshot_read_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_upstream_output_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Upstream - table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_backfill_upstream_output_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"We first record the total blocking duration(ns) of output buffer of each actor. It shows how much time it takes an actor to process a message, i.e. a barrier, a watermark or rows of data, on average. Then we divide this duration by 1 second and show it as a percentage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":43,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}->{{downstream_fragment_id}}","metric":"","query":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Blocking Time Ratio (Backpressure)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":55},"height":null,"hideTimeOverride":false,"id":44,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":45,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","query":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Running query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":46,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","query":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Rejected query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":47,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","query":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Completed query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":48,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":49,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Local Execution Mode","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":50,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":51,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_type}} @ {{source_id}}","metric":"","query":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":52,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector_type}} @ {{sink_id}}","metric":"","query":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Sink Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Connector Node","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, instance)","description":"Reporting instance of the metric","hide":0,"includeAll":true,"label":"Node","multi":true,"name":"node","options":[],"query":{"query":"label_values(process_cpu_seconds_total, instance)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, job)","description":"Reporting job of the metric","hide":0,"includeAll":true,"label":"Job","multi":true,"name":"job","options":[],"query":{"query":"label_values(process_cpu_seconds_total, job)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"}]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dashboard","uid":"Fcy3uV1nz","version":0} +{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dashboard","editable":true,"gnetId":null,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Actor/Table Id Info","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from actor id to fragment id","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Id Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from materialized view table id to it's internal table ids","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":12,"y":1},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id, table_name, table_type)","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Overview","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":10},"height":null,"hideTimeOverride":false,"id":5,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (source_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{source_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregated Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":10},"height":null,"hideTimeOverride":false,"id":6,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source_id {{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregated Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":7,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":18},"height":null,"hideTimeOverride":false,"id":8,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The time that the data between two consecutive barriers gets fully processed, i.e. the computation results are made durable into materialized views or sink to external systems. This metric shows to users the freshness of materialized views.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":9,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Alerts in the system group by type:\n - Too Many Barriers: there are too many uncommitted barriers generated. This means the streaming graph is stuck or under heavy load. Check 'Barrier Latency' panel.\n - Recovery Triggered: cluster recovery is triggered. Check 'Errors by Type' / 'Node Count' panels.\n - Lagging Version: the checkpointed or pinned version id is lagging behind the current version id. Check 'Hummock Manager' section in dev dashboard.\n - Lagging Epoch: the pinned or safe epoch is lagging behind the current max committed epoch. Check 'Hummock Manager' section in dev dashboard.\n - Lagging Compaction: there are too many files in L0. This can be caused by compactor failure or lag of compactor resource. Check 'Compaction' section in dev dashboard.\n - Lagging Vacuum: there are too many stale files waiting to be cleaned. This can be caused by compactor failure or lag of compactor resource. Check 'Compaction' section in dev dashboard.\n - Abnormal Meta Cache Memory: the meta cache memory usage is too large, exceeding the expected 10 percent.\n - Abnormal Block Cache Memory: the block cache memory usage is too large, exceeding the expected 10 percent.\n - Abnormal Uploading Memory Usage: uploading memory is more than 70 percent of the expected, and is about to spill.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":26},"height":null,"hideTimeOverride":false,"id":10,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"} >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Too Many Barriers","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) > bool 0 + sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) > bool 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Recovery Triggered","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100) + ((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Version","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"} == 0) + ((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_safe_epoch{job=~\"$job\",instance=~\"$node\"} == 0)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Epoch","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(label_replace(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}, 'L0', 'L0', 'level_index', '.*_L0') unless storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (L0) >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Compaction","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"} >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Vacuum","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_meta_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Meta Cache Memory","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_block_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Block Cache Memory","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_uploading_memory_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 0.7","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Uploading Memory Usage","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Alerts","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Errors in the system group by type","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":34},"height":null,"hideTimeOverride":false,"id":11,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compute error {{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"parse error {{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"source_status_is_up{job=~\"$job\",instance=~\"$node\"} == 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source error: source_id={{source_id}}, source_name={{source_name}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote storage error {{type}}: {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Errors","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":34},"height":null,"hideTimeOverride":false,"id":12,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Local mode","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distributed mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Query QPS","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of each type of RisingWave components alive.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":42},"height":null,"hideTimeOverride":false,"id":13,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of active sessions in frontend nodes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":42},"height":null,"hideTimeOverride":false,"id":14,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Active Sessions","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":50},"height":null,"hideTimeOverride":false,"id":15,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The CPU usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":16,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of CPU cores per RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":17,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU Core Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"CPU","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":51},"height":null,"hideTimeOverride":false,"id":18,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The memory usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":19,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":20,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Memory Usage (Total)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":21,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(actor_memory_usage[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"streaming actor - {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage meta cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage block cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage write buffer - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized_view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Memory Usage (Detailed)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Executor cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":22,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Join - cache miss - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Join - total lookups - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n appendonly - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n appendonly - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lookup executor - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lookup executor - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal join - cache miss - table_id {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal join - total lookups - table_id {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Materialize - cache hit count - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Materialize - total cache count - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":23,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"join executor cache miss ratio - - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n appendonly cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream lookup cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream temporal join cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialize executor cache miss ratio - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Miss Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":24,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"memory cache - {{table_id}} @ {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total_meta_miss_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage bloom filter statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":25,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_check_bloom_filter_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter total - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_positive_but_non_exist_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Bloom Filer","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage file cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":26,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage File Cache","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":52},"height":null,"hideTimeOverride":false,"id":27,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Send/Recv throughput per node for streaming exchange","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":28,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Send @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Recv @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Streming Remote Exchange (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The remote storage read/write throughput per node","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":29,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Remote I/O (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":30,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Exchange Recv (Rows/s)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Network","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":53},"height":null,"hideTimeOverride":false,"id":31,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\n Objects are classified into 3 groups:\n - not referenced by versions: these object are being deleted from object store.\n - referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n - referenced by current version: these objects are in the latest version.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":32,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The storage size of each materialized view","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":33,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_materialized_view_stats{metric='materialized_view_total_size',job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{metric}}, mv id - {{table_id}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\n Objects are classified into 3 groups:\n - not referenced by versions: these object are being deleted from object store.\n - referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n - referenced by current version: these objects are in the latest version.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":34,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of bytes that have been written by compaction.Flush refers to the process of compacting Memtables to SSTables at Level 0.Compaction refers to the process of compacting SSTables at one level to another level.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":35,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Compaction - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The remote storage read/write throughput","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":36,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Remote I/O (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Size statistics for checkpoint","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":37,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":54},"height":null,"hideTimeOverride":false,"id":38,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":39,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":40,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized executor actor per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":41,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_executor_row_count{executor_identity=~\".*MaterializeExecutor.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) * on(actor_id) group_left(materialized_view_id, table_name) (group(table_info{table_type=~\"MATERIALIZED_VIEW\",job=~\"$job\",instance=~\"$node\"}) by (actor_id, materialized_view_id, table_name))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized view {{table_name}} table_id {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been read from the backfill operator used by MV on MV","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":42,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_snapshot_read_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Read Snapshot - table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_upstream_output_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Upstream - table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"We first record the total blocking duration(ns) of output buffer of each actor. It shows how much time it takes an actor to process a message, i.e. a barrier, a watermark or rows of data, on average. Then we divide this duration by 1 second and show it as a percentage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":43,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}->{{downstream_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Blocking Time Ratio (Backpressure)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":55},"height":null,"hideTimeOverride":false,"id":44,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":45,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Running query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":46,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Rejected query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":47,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Completed query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":48,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":49,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Local Execution Mode","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":50,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":51,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_type}} @ {{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":52,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector_type}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Sink Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Connector Node","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, instance)","description":"Reporting instance of the metric","hide":0,"includeAll":true,"label":"Node","multi":true,"name":"node","options":[],"query":{"query":"label_values(process_cpu_seconds_total, instance)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, job)","description":"Reporting job of the metric","hide":0,"includeAll":true,"label":"Job","multi":true,"name":"job","options":[],"query":{"query":"label_values(process_cpu_seconds_total, job)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"}]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dashboard","uid":"Fcy3uV1nz","version":0} diff --git a/grafana/risingwave-dev-dashboard.dashboard.py b/grafana/risingwave-dev-dashboard.dashboard.py index a95911760dd85..335557fc73d75 100644 --- a/grafana/risingwave-dev-dashboard.dashboard.py +++ b/grafana/risingwave-dev-dashboard.dashboard.py @@ -2346,6 +2346,16 @@ def section_hummock_write(outer_panels): ), ], ), + panels.timeseries_count( + "Mem Table Spill Count", + "", + [ + panels.target( + f"sum(irate({table_metric('state_store_mem_table_spill_counts')}[$__rate_interval])) by ({COMPONENT_LABEL},{NODE_LABEL},table_id)", + "mem table spill table id - {{table_id}} @ {{%s}}" % NODE_LABEL, + ), + ], + ), panels.timeseries_bytes( "Checkpoint Sync Size", "", diff --git a/grafana/risingwave-dev-dashboard.json b/grafana/risingwave-dev-dashboard.json index ab2d61b963b23..648ef62c32e22 100644 --- a/grafana/risingwave-dev-dashboard.json +++ b/grafana/risingwave-dev-dashboard.json @@ -1 +1 @@ -{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dev Dashboard","editable":true,"gnetId":null,"graphTooltip":0,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from actor id to fragment id","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"actor_info{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Id Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from materialized view table id to it's internal table ids","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"table_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"table_info{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Actor/Table Id Info","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of each type of RisingWave components alive.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":5,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","query":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The memory usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":6,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The CPU usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":7,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cpu usage (total) - {{job}} @ {{instance}}","metric":"","query":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cpu usage (avg per core) - {{job}} @ {{instance}}","metric":"","query":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"RW cluster can configure multiple meta nodes to achieve high availability. One is the leader and the rest are the followers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":8,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(meta_num{job=~\"$job\",instance=~\"$node\"}) by (worker_addr,role)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_addr}} @ {{role}}","metric":"","query":"sum(meta_num{job=~\"$job\",instance=~\"$node\"}) by (worker_addr,role)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Meta Cluster","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Cluster Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":2},"height":null,"hideTimeOverride":false,"id":9,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The rate of successful recovery attempts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":10,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","query":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Recovery Successful Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of failed reocovery attempts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":11,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","query":"sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Failed recovery attempts","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Time spent in a successful recovery attempt","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":12,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency pmax - {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by (le) (rate(recovery_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by (le) (rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency avg","metric":"","query":"sum by (le) (rate(recovery_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by (le) (rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Recovery latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Recovery","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":3},"height":null,"hideTimeOverride":false,"id":13,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":14,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Each query is executed in parallel with a user-defined parallelism. This figure shows the throughput of each parallelism. The throughput of all the parallelism added up is equal to Source Throughput(rows).","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":15,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(partition_input_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","query":"rate(partition_input_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":16,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","query":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Each query is executed in parallel with a user-defined parallelism. This figure shows the throughput of each parallelism. The throughput of all the parallelism added up is equal to Source Throughput(MB/s).","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":17,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","query":"(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"RisingWave ingests barriers periodically to trigger computation and checkpoints. The frequency of barrier can be set by barrier_interval_ms. This metric shows how many rows are ingested between two consecutive barriers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":18,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_rows_per_barrier_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} @ {{instance}}","metric":"","query":"rate(stream_source_rows_per_barrier_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) per barrier","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Monitor each source upstream, 0 means the upstream is not normal, 1 means the source is ready.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":19,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"source_status_is_up{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source_id={{source_id}}, source_name={{source_name}} @ {{instance}}","metric":"","query":"source_status_is_up{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Upstream Status","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Source Split Change Events frequency by source_id and actor_id","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":20,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_split_change_event_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_source_split_change_event_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Split Change Events frequency(events/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Kafka Consumer Lag Size by source_id, partition and actor_id","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":21,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"high_watermark{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}} partition={{partition}}","metric":"","query":"high_watermark{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"latest_message_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}} partition={{partition}} actor_id={{actor_id}}","metric":"","query":"latest_message_id{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kafka Consumer Lag Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":22,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}}","metric":"","query":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":23,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id, actor_id) * on(actor_id) group_left(sink_name) sink_info{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}} - actor {{actor_id}}","metric":"","query":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id, actor_id) * on(actor_id) group_left(sink_name) sink_info{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s) per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":24,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}}","metric":"","query":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":25,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (actor_id, table_id) * on(actor_id, table_id) group_left(table_name) table_info{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}} - actor {{actor_id}}","metric":"","query":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (actor_id, table_id) * on(actor_id, table_id) group_left(table_name) table_info{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s) per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been read from the backfill snapshot","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":26,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_snapshot_read_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_backfill_snapshot_read_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Snapshot Read Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been output from the backfill upstream","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":27,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_upstream_output_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_backfill_upstream_output_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Upstream Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of barriers that have been ingested but not completely processed. This metric reflects the current level of congestion within the system.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":28,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all_barrier","metric":"","query":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"in_flight_barrier_nums{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"in_flight_barrier","metric":"","query":"in_flight_barrier_nums{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The duration between the time point when the scheduled barrier needs to be sent and the time point when the barrier gets actually sent to all the compute nodes. Developers can thus detect any internal congestion.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":29,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_send_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_avg","metric":"","query":"rate(meta_barrier_send_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Send Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The time that the data between two consecutive barriers gets fully processed, i.e. the computation results are made durable into materialized views or sink to external systems. This metric shows to users the freshness of materialized views.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":30,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","query":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":31,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_avg","metric":"","query":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier In-Flight Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":32,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p999 - {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_pmax - {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_avg - {{instance}}","metric":"","query":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Sync Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":33,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_wait_commit_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_avg","metric":"","query":"rate(meta_barrier_wait_commit_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Wait Commit Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of actors that have processed the earliest in-flight barriers per second. This metric helps users to detect potential congestion or stuck in the system.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":34,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_barrier_manager_progress{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","query":"rate(stream_barrier_manager_progress{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Earliest In-Flight Barrier Progress","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":4},"height":null,"hideTimeOverride":false,"id":35,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"We first record the total blocking duration(ns) of output buffer of each actor. It shows how much time it takes an actor to process a message, i.e. a barrier, a watermark or rows of data, on average. Then we divide this duration by 1 second and show it as a percentage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":36,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}->{{downstream_fragment_id}}","metric":"","query":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Blocking Time Ratio (Backpressure)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":37,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_input_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, upstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}<-{{upstream_fragment_id}}","metric":"","query":"avg(rate(stream_actor_input_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, upstream_fragment_id) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":38,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","query":"rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":39,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","query":"rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The operator-level memory usage statistics collected by each LRU cache","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":40,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (table_id, desc)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} desc: {{desc}}","metric":"","query":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (table_id, desc)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_memory_usage{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} actor {{actor_id}} desc: {{desc}}","metric":"","query":"stream_memory_usage{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Memory Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Memory usage aggregated by materialized views","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":41,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized view {{materialized_view_id}}","metric":"","query":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Memory Usage of Materialized Views","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":42,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"temporal join cache miss, table_id {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"temporal join cache miss, table_id {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Temporal Join Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":43,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache hit count - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total cached count - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache hit count - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total cached count - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialize Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":44,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache lookup count - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss count - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache lookup count - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss count - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Over Window Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":45,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"join executor cache miss ratio - - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} fragment {{fragment_id}}","metric":"","query":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n appendonly cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream lookup cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream temporal join cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialize executor cache miss ratio - table {{table_id}} fragment {{fragment_id}} {{instance}}","metric":"","query":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_over_window_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_over_window_cache_lookup_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Over window cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","query":"(sum(rate(stream_over_window_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_over_window_cache_lookup_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Miss Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":46,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, fragment_id, wait_side, job)(rate(stream_join_barrier_align_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le,fragment_id,wait_side,job) (rate(stream_join_barrier_align_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","query":"sum by(le, fragment_id, wait_side, job)(rate(stream_join_barrier_align_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le,fragment_id,wait_side,job) (rate(stream_join_barrier_align_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Barrier Align","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":47,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","query":"rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":48,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}} {{side}}","metric":"","query":"sum(rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Match Duration Per Second","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Multiple rows with distinct primary keys may have the same join key. This metric counts the number of join keys in the executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":49,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (fragment_id, side)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}} {{side}}","metric":"","query":"sum(stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (fragment_id, side)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}} {{side}}","metric":"","query":"stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of matched rows on the opposite side","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":50,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, actor_id, table_id) (rate(stream_join_matched_join_keys_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, fragment_id, table_id) (rate(stream_join_matched_join_keys_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","query":"sum by(le, job, actor_id, table_id) (rate(stream_join_matched_join_keys_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, fragment_id, table_id) (rate(stream_join_matched_join_keys_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Matched Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":51,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups - table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss - table {{table_id}} actor {{actor_id}}}","metric":"","query":"rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each StreamChunk","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":52,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg cached keys count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg distinct cached keys count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg cached keys count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg distinct cached keys count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of dirty (unflushed) groups in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":53,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Dirty Groups Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The total heap size of dirty (unflushed) groups in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":54,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups heap size | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups heap size | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Dirty Groups Heap Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in each top_n executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":55,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n appendonly cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n cached count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n appendonly cached count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"TopN Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in temporal join executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":56,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal Join cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal Join cached count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Temporal Join Cache Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in lookup executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":57,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lookup cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lookup cached count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lookup Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in over window executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":58,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"over window cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","query":"sum(stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"over window cached count | table {{table_id}} actor {{actor_id}}","metric":"","query":"stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Over Window Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"When enabled, this metric shows the input throughput of each executor.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":59,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_identity, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_identity}} fragment {{fragment_id}}","metric":"","query":"sum(rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_identity, fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_identity}} actor {{actor_id}}","metric":"","query":"rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The actor-level memory usage statistics reported by TaskLocalAlloc. (Disabled by default)","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":60,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(actor_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","query":"sum(actor_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (fragment_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_memory_usage{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","query":"actor_memory_usage{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Memory Usage (TaskLocalAlloc)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":5},"height":null,"hideTimeOverride":false,"id":61,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":62,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_actor_execution_time{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_actor_execution_time{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Execution Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":63,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":64,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":8},"height":null,"hideTimeOverride":false,"id":65,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":8},"height":null,"hideTimeOverride":false,"id":66,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":67,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":16},"height":null,"hideTimeOverride":false,"id":68,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":16},"height":null,"hideTimeOverride":false,"id":69,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":70,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":24},"height":null,"hideTimeOverride":false,"id":71,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":24},"height":null,"hideTimeOverride":false,"id":72,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":73,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":32},"height":null,"hideTimeOverride":false,"id":74,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":32},"height":null,"hideTimeOverride":false,"id":75,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":76,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":40},"height":null,"hideTimeOverride":false,"id":77,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Avg Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors (Tokio)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":6},"height":null,"hideTimeOverride":false,"id":78,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":79,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","query":"rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Send Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":80,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","query":"rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Recv Throughput","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Exchange","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":7},"height":null,"hideTimeOverride":false,"id":81,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":82,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","query":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","query":"sum(user_compute_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compute Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":83,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","query":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","query":"sum(user_source_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":84,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_reader_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: actor_id={{actor_id}}, source_id={{source_id}})","metric":"","query":"sum(user_source_reader_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_reader_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: actor_id={{actor_id}}, source_id={{source_id}})","metric":"","query":"sum(user_source_reader_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Reader Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":85,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_sink_error{job=~\"$job\",instance=~\"$node\"}) by (connector_name, executor_id, error_msg)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{connector_name}}: {{error_msg}} ({{executor_id}})","metric":"","query":"sum(user_sink_error{job=~\"$job\",instance=~\"$node\"}) by (connector_name, executor_id, error_msg)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink by Connector","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"User Streaming Errors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":86,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":87,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","query":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Exchange Recv Row Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":88,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_task_num{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mpp Task Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"All memory usage of batch executors in bytes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":89,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_total_mem{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_total_mem{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mem Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":90,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_heartbeat_worker_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_heartbeat_worker_num{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Heartbeat Worker Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the memory usage of mem_table.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":91,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table size total - {{job}} @ {{instance}}","metric":"","query":"sum(state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table size - table id {{table_id}} instance id {{instance_id}} {{job}} @ {{instance}}","metric":"","query":"state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Mem Table Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the item counts in mem_table.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":92,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table counts total - {{job}} @ {{instance}}","metric":"","query":"sum(state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table count - table id {{table_id}} instance id {{instance_id}} {{job}} @ {{instance}}","metric":"","query":"state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Mem Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":93,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Row SeqScan Next Duration","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":94,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":95,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{table_id}} @ {{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_sst_store_block_request_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total_meta_miss_count - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Hummock has three parts of memory usage: 1. Meta Cache 2. Block CacheThis metric shows the real memory usage of each of these three caches.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":96,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache - {{job}} @ {{instance}}","metric":"","query":"avg(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"data cache - {{job}} @ {{instance}}","metric":"","query":"avg(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":97,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"block cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Miss Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":98,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_scan_key_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type, table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{table_id}} @ {{type}} @ {{instance}}","metric":"","query":"sum(rate(state_store_iter_scan_key_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type, table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Iter keys flow","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":99,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p50 - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p99 - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts pmax - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Merged SSTs","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of the latency of Get operations that have been issued to the state store.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":100,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of the time spent on iterator initialization.Histogram of the time spent on iterator scanning.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":101,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_init_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_init_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_iter_init_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_init_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_scan_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_scan_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_iter_scan_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_scan_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":102,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive count - {{table_id}} - {{type}}","metric":"","query":"sum(irate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter positive count - {{table_id}} - {{type}}","metric":"","query":"sum(irate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter check count- {{table_id}} - {{type}}","metric":"","query":"sum(irate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Positive / Total","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":103,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter positive rate - {{table_id}} - {{type}}","metric":"","query":"(sum(rate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter Positive Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"False-Positive / Total","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":104,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(((sum(rate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter false positive rate - {{table_id}} - {{type}}","metric":"","query":"(((sum(rate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter False-Positive Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":105,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_iter_slow_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"state_store_iter_slow_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Slow Fetch Meta Unhits","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":106,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_shared_buffer_hit_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer hit - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_shared_buffer_hit_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_in_process_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_iter_in_process_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":107,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":108,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":109,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read p50 - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read p99 - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read pmax - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Read Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":110,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Count - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size of a single key-value pair when reading by operation Get.Operation Get gets a single key-value pair with respect to a caller-specified key. If the key does not exist in the storage, the size of key is counted into this metric and the size of value is 0.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":111,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size of all the key-value paris when reading by operation Iter.Operation Iter scans a range of key-value pairs.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":112,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_iter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":113,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":114,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_iter_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"state_store_iter_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Unhits","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock (Read)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":10},"height":null,"hideTimeOverride":false,"id":115,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the real memory usage of uploader.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":116,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading memory - {{job}} @ {{instance}}","metric":"","query":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_uploader_uploading_task_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading task size - {{job}} @ {{instance}}","metric":"","query":"sum(state_store_uploader_uploading_task_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader Memory Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of time spent on compacting shared buffer to remote storage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":117,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(state_store_sync_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Build and Sync Sstable Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":118,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.5, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write p50 - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(0.5, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.99, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write p99 - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(0.99, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write pmax - materialized view {{materialized_view_id}}","metric":"","query":"sum(histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Write Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":119,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_merge_imm_task_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"merge imm tasks - {{table_id}} @ {{instance}}","metric":"","query":"sum(irate(state_store_merge_imm_task_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_spill_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Uploader spill tasks - {{uploader_stage}} @ {{instance}}","metric":"","query":"sum(irate(state_store_spill_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader - Tasks Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":120,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_merge_imm_memory_sz{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Merging tasks memory size - {{table_id}} @ {{instance}}","metric":"","query":"sum(rate(state_store_merge_imm_memory_sz{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_spill_task_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Uploading tasks size - {{uploader_stage}} @ {{instance}}","metric":"","query":"sum(rate(state_store_spill_task_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader - Task Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":121,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write batch - {{table_id}} @ {{job}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"l0 - {{job}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":122,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":123,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_write_batch_tuple_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write_batch_kv_pair_count - {{table_id}} @ {{instance}}","metric":"","query":"sum(irate(state_store_write_batch_tuple_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Item Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":124,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_size_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id) / sum(rate(state_store_write_batch_size_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_write_batch_size_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id) / sum(rate(state_store_write_batch_size_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_shared_buffer_to_sstable_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance) / sum(rate(compactor_shared_buffer_to_sstable_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sync - {{job}} @ {{instance}}","metric":"","query":"sum(rate(compactor_shared_buffer_to_sstable_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance) / sum(rate(compactor_shared_buffer_to_sstable_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the statistics of mem_table size on flush. By default only max (p100) is shown.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":125,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, table_id, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, table_id, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_write_batch_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, table_id, job, instance) (rate(state_store_write_batch_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(state_store_write_batch_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, table_id, job, instance) (rate(state_store_write_batch_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Batch Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":126,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Sync Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock (Write)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":11},"height":null,"hideTimeOverride":false,"id":127,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of SSTables at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":128,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"sum(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SSTable Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size(KB) of SSTables at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":129,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SSTable Size(KB)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The of bytes that have been written by commit epoch per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":130,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_commit_write_throughput{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{table_id}}","metric":"","query":"sum(rate(storage_commit_write_throughput{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Commit Flush Bytes by Table","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have completed or failed","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":131,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency{result!='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","query":"sum(storage_level_compact_frequency{result!='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Failure Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have completed or failed","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":132,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency{result='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","query":"sum(storage_level_compact_frequency{result='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Success Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have been skipped.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":133,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_skip_compact_frequency{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (level, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{level}}-{{type}}","metric":"","query":"sum(rate(storage_skip_compact_frequency{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (level, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Skip Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg l0 select_level_count of the compact task, and categorize it according to different cg, levels and task types","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":134,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, group, type)(irate(storage_l0_compact_level_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_l0_compact_level_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg cg{{group}}@{{type}}","metric":"","query":"sum by(le, group, type)(irate(storage_l0_compact_level_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_l0_compact_level_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task L0 Select Level Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg file count of the compact task, and categorize it according to different cg, levels and task types","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":135,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, group, type)(irate(storage_compact_task_file_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_compact_task_file_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg cg{{group}}@{{type}}","metric":"","query":"sum by(le, group, type)(irate(storage_compact_task_file_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_compact_task_file_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task File Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The distribution of the compact task size triggered, including p90 and max. and categorize it according to different cg, levels and task types.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":136,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - cg{{group}}@{{type}}","metric":"","query":"histogram_quantile(0.9, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - cg{{group}}@{{type}}","metric":"","query":"histogram_quantile(1.0, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task Size Distribution","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that are running.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":137,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(storage_compact_task_pending_num{job=~\"$job\",instance=~\"$node\"}) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor_task_split_count - {{job}} @ {{instance}}","metric":"","query":"avg(storage_compact_task_pending_num{job=~\"$job\",instance=~\"$node\"}) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Running Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"compact-task: The total time have been spent on compaction.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":138,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p50 - {{job}}","metric":"","query":"histogram_quantile(0.5, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p90 - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task pmax - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range p90 - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range pmax - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id p90 - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id pmax - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{job}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{job}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(compute_refill_cache_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compute_apply_version_duration_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(compute_refill_cache_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(compactor_compact_task_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task avg","metric":"","query":"sum by(le)(rate(compactor_compact_task_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(state_store_compact_sst_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range avg","metric":"","query":"sum by(le)(rate(state_store_compact_sst_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"KBs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":139,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job) + sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}}","metric":"","query":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job) + sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","query":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","query":"sum(rate(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_fast_compact_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fast compact - {{job}}","metric":"","query":"sum(rate(compactor_fast_compact_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of bytes that have been written by compaction.Flush refers to the process of compacting Memtables to SSTables at Level 0.Write refers to the process of compacting SSTables at one level to another level.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":140,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","query":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","query":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Bytes(GiB)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Write amplification is the amount of bytes written to the remote storage by compaction for each one byte of flushed SSTable data. Write amplification is by definition higher than 1.0 because we write each piece of data to L0, and then write it again to an SSTable, and then compaction may read this piece of data and write it to a new SSTable, that's another write.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":141,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) / sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"})","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write amplification","metric":"","query":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) / sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"})","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Amplification","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of SSTables that is being compacted at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":142,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"storage_level_compact_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting SSTable Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"num of compact_task","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":143,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_task_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task}}","metric":"","query":"storage_level_compact_task_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":144,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from next level","metric":"","query":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from current level","metric":"","query":"sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} write to next level","metric":"","query":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read/Write by Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":145,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_write_sstn{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} write to next level","metric":"","query":"sum(irate(storage_level_compact_write_sstn{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_read_sstn_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from next level","metric":"","query":"sum(irate(storage_level_compact_read_sstn_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_read_sstn_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from current level","metric":"","query":"sum(irate(storage_level_compact_read_sstn_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read/Write by level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total bytes gotten from sstable_bloom_filter, for observing bloom_filter size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":146,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_meta - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_file - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total bytes gotten from sstable_avg_key_size, for observing sstable_avg_key_size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":147,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_key_size - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_value_size - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Item Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg count gotten from sstable_distinct_epoch_count, for observing sstable_distinct_epoch_count","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":148,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_epoch_count - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Stat","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total time of operations which read from remote storage when enable prefetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":149,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Remote Read Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":150,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_iter_scan_key_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{type}} @ {{instance}} ","metric":"","query":"sum(rate(compactor_iter_scan_key_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Iter keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"bytes of Lsm tree needed to reach balance","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":151,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_pending_bytes{job=~\"$job\",instance=~\"$node\"}) by (instance, group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact pending bytes - {{group}} @ {{instance}} ","metric":"","query":"sum(storage_compact_pending_bytes{job=~\"$job\",instance=~\"$node\"}) by (instance, group)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Compact Pending Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"compression ratio of each level of the lsm tree","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":152,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_level_compression_ratio{job=~\"$job\",instance=~\"$node\"}) by (instance, group, level, algorithm)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lsm compression ratio - cg{{group}} @ L{{level}} - {{algorithm}} {{instance}}","metric":"","query":"sum(storage_compact_level_compression_ratio{job=~\"$job\",instance=~\"$node\"}) by (instance, group, level, algorithm)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Level Compression Ratio","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Compaction","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":12},"height":null,"hideTimeOverride":false,"id":153,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":154,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":155,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":156,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type!~'streaming_upload_write_bytes|streaming_read_read_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count{type!~'streaming_upload_write_bytes|streaming_read_read_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'upload|delete',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count{type=~'upload|delete',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":157,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":158,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Failure Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":159,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(aws_sdk_retry_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(irate(aws_sdk_retry_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(s3_read_request_retry_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(irate(s3_read_request_retry_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Retry Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"There are two types of operations: 1. GET, SELECT, and DELETE, they cost 0.0004 USD per 1000 requests. 2. PUT, COPY, POST, LIST, they cost 0.005 USD per 1000 requests.Reading from S3 across different regions impose extra cost. This metric assumes 0.01 USD per 1GB data transfer. Please checkout AWS's pricing model for more accurate calculation.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":160,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}) * 0.01 / 1000 / 1000 / 1000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"(Cross Region) Data Transfer Cost","metric":"","query":"sum(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}) * 0.01 / 1000 / 1000 / 1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete',job=~\"$job\",instance=~\"$node\"}) * 0.0004 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GET, SELECT, and all other Requests Cost","metric":"","query":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete',job=~\"$job\",instance=~\"$node\"}) * 0.0004 / 1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list',job=~\"$job\",instance=~\"$node\"}) * 0.005 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"PUT, COPY, POST, LIST Requests Cost","metric":"","query":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list',job=~\"$job\",instance=~\"$node\"}) * 0.005 / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Realtime)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric uses the total size of data in S3 at this second to derive the cost of storing data for a whole month. The price is 0.023 USD per GB. Please checkout AWS's pricing model for more accurate calculation.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":161,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance) * 0.023 / 1000 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Monthly Storage Cost","metric":"","query":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance) * 0.023 / 1000 / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Monthly)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Object Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":13},"height":null,"hideTimeOverride":false,"id":162,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":163,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache {{op}} {{extra}} @ {{instance}}","metric":"","query":"sum(rate(foyer_storage_op_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":164,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":165,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","query":"sum(rate(foyer_storage_op_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":166,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(foyer_storage_total_bytes{job=~\"$job\",instance=~\"$node\"}) by (foyer, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} size @ {{instance}}","metric":"","query":"sum(foyer_storage_total_bytes{job=~\"$job\",instance=~\"$node\"}) by (foyer, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":167,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) / (sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) + sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"miss\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache hit ratio @ {{instance}}","metric":"","query":"sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) / (sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) + sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"miss\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Hit Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":168,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(refill_queue_total) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"refill queue length @ {{instance}}","metric":"","query":"sum(refill_queue_total) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Queue Length","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":169,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(refill_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} file cache refill - {{op}} @ {{instance}}","metric":"","query":"sum(rate(refill_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(refill_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} file cache refill - {{op}} @ {{instance}}","metric":"","query":"sum(rate(refill_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":170,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Tiered Cache","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":14},"height":null,"hideTimeOverride":false,"id":171,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":172,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p50 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p99 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time pmax - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lock Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":173,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p50 - {{method}}","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p99 - {{method}}","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time pmax - {{method}}","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Real Process Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":174,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version size","metric":"","query":"storage_version_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":175,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"current version id","metric":"","query":"storage_current_version_id{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"checkpoint version id","metric":"","query":"storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned version id","metric":"","query":"storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_safepoint_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min safepoint version id","metric":"","query":"storage_min_safepoint_version_id{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Id","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":176,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"max committed epoch","metric":"","query":"storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"safe epoch","metric":"","query":"storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned epoch","metric":"","query":"storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":177,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_key_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_value_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_value_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":178,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_materialized_view_stats{metric='materialized_view_total_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{metric}}, mv id - {{table_id}} ","metric":"","query":"storage_materialized_view_stats{metric='materialized_view_total_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":179,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_count',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_key_count',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\nObjects are classified into 3 groups:\n- not referenced by versions: these object are being deleted from object store.\n- referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n- referenced by current version: these objects are in the latest version.\n\nAdditionally, a metric on all objects (including dangling ones) is updated with low-frequency. The metric is updated right before full GC. So subsequent full GC may reduce the actual value significantly, without updating the metric.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":180,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","query":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","query":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","query":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_total_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all objects (including dangling ones)","metric":"","query":"storage_total_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Refer to `Object Total Number` panel for classification of objects.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":181,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","query":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","query":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","query":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_total_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all objects, including dangling ones","metric":"","query":"storage_total_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"total number of hummock version delta log","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":182,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_delta_log_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"delta log total number","metric":"","query":"storage_delta_log_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Delta Log Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"hummock version checkpoint latency","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":183,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(storage_version_checkpoint_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(storage_version_checkpoint_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_avg","metric":"","query":"rate(storage_version_checkpoint_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(storage_version_checkpoint_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Checkpoint Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"When certain per compaction group threshold is exceeded (e.g. number of level 0 sub-level in LSMtree), write op to that compaction group is stopped temporarily. Check log for detail reason of write stop.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":184,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_write_stop_compaction_groups{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compaction_group_{{compaction_group_id}}","metric":"","query":"storage_write_stop_compaction_groups{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Stop Compaction Groups","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"total number of attempts to trigger full GC","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":185,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_full_gc_trigger_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"full_gc_trigger_count","metric":"","query":"storage_full_gc_trigger_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Full GC Trigger Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"the object id watermark used in last full GC","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":186,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_full_gc_last_object_id_watermark{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"full_gc_last_object_id_watermark","metric":"","query":"storage_full_gc_last_object_id_watermark{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Full GC Last Watermark","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":187,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Event Loop Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The times of move_state_table occurs","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":188,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_move_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"move table cg{{group}}","metric":"","query":"sum(storage_move_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}) by (group)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Move State Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of state_tables in each CG","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":189,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"state table cg{{group}}","metric":"","query":"sum(irate(storage_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"State Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of branched_sst in each CG","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":190,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_branched_sst_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"branched sst cg{{group}}","metric":"","query":"sum(irate(storage_branched_sst_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Branched SST Count","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":15},"height":null,"hideTimeOverride":false,"id":191,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total backup job count since the Meta node starts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":192,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"backup_job_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"job count","metric":"","query":"backup_job_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Latency of backup jobs since the Meta node starts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":193,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p50 - {{state}}","metric":"","query":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p99 - {{state}}","metric":"","query":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time pmax - {{state}}","metric":"","query":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Process Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Backup Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":194,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":195,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":196,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Drop latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":197,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetCatalog latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Catalog Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":17},"height":null,"hideTimeOverride":false,"id":198,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":199,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"AddWorkerNode latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":200,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ListAllNodes latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Cluster Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":201,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":202,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"CreateMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":203,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"DropMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":204,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Flush latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Stream Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":19},"height":null,"hideTimeOverride":false,"id":205,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":206,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinVersionBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":207,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinSnapshotBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":208,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ReportCompactionTasks latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":209,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetNewSstIds latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":20},"height":null,"hideTimeOverride":false,"id":210,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":211,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_counts - {{instance}}","metric":"","query":"sum(irate(state_store_report_compaction_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":212,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_version_before_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_avg","metric":"","query":"sum(irate(state_store_unpin_version_before_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"version_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":213,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latencyp90 - {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_avg","metric":"","query":"sum(irate(state_store_pin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_avg","metric":"","query":"sum(irate(state_store_unpin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":214,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_counts - {{instance}}","metric":"","query":"sum(irate(state_store_pin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_counts - {{instance}}","metric":"","query":"sum(irate(state_store_unpin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":215,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_avg","metric":"","query":"sum(irate(state_store_get_new_sst_ids_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":216,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_counts - {{instance}}","metric":"","query":"sum(irate(state_store_get_new_sst_ids_latency_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":217,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_avg","metric":"","query":"sum(irate(state_store_report_compaction_task_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC: Hummock Meta Client","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":21},"height":null,"hideTimeOverride":false,"id":218,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of active sessions","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":219,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Active Sessions","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":220,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per Second (Local Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":221,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per Second (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":222,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","query":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Running Queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":223,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","query":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Rejected queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":224,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","query":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Completed Queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":225,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":226,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency (Local Query Mode)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Frontend","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":22},"height":null,"hideTimeOverride":false,"id":227,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":228,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(lru_runtime_loop_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(lru_runtime_loop_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager loop count per sec","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":229,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_watermark_step{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_watermark_step{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark steps","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"watermark_time is the current lower watermark of cached data. physical_now is the current time of the machine. The diff (physical_now - watermark_time) shows how much data is cached.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":230,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_physical_now_ms{job=~\"$job\",instance=~\"$node\"} - lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_physical_now_ms{job=~\"$job\",instance=~\"$node\"} - lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between watermark_time and now (ms)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":231,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jemalloc_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The allocated memory of jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":232,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_active_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jemalloc_active_bytes{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The active memory of jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":233,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jvm_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jvm_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The allocated memory of jvm","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":234,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jvm_active_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jvm_active_bytes{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The active memory of jvm","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":235,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"} - on() group_right() lru_evicted_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} actor {{actor_id}} desc: {{desc}}","metric":"","query":"lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"} - on() group_right() lru_evicted_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between current watermark and evicted watermark time (ms) for actors","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":23},"height":null,"hideTimeOverride":false,"id":236,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":237,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_type}} @ {{source_id}}","metric":"","query":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":238,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector_type}} @ {{sink_id}}","metric":"","query":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Sink Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Connector Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":239,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":240,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 @ {{connector}} {{sink_id}}","metric":"","query":"histogram_quantile(0.5, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 @ {{connector}} {{sink_id}}","metric":"","query":"histogram_quantile(0.99, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax @ {{connector}} {{sink_id}}","metric":"","query":"histogram_quantile(1.0, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, connector, sink_id)(rate(sink_commit_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(sink_commit_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{connector}} @ {{sink_id}}","metric":"","query":"sum by(le, connector, sink_id)(rate(sink_commit_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(sink_commit_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Commit Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":241,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"latest write epoch @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","query":"log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"latest read epoch @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","query":"log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Read/Write Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":242,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(max(log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Consume lag @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","query":"(max(log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Lag","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":243,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"clamp_min((max(log_store_first_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000, 0)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Consume persistent log lag @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","query":"clamp_min((max(log_store_first_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000, 0)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Consume Persistent Log Lag","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":244,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}}","metric":"","query":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Consume Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":245,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}} @ {{executor_id}} {{instance}}","metric":"","query":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Log Store Consume Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":246,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}}","metric":"","query":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Write Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":247,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}} @ {{executor_id}} {{instance}}","metric":"","query":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Log Store Write Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":248,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_read_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","query":"sum(rate(kv_log_store_storage_read_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Read Storage Row Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":249,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_read_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","query":"sum(rate(kv_log_store_storage_read_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Read Storage Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":250,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_write_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","query":"sum(rate(kv_log_store_storage_write_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Write Storage Row Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":251,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_write_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","query":"sum(rate(kv_log_store_storage_write_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Write Storage Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Sink Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":25},"height":null,"hideTimeOverride":false,"id":252,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Current number of messages in producer queues","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":253,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","query":"rdkafka_top_msg_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Count in Producer Queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Current total size of messages in producer queues","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":254,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_msg_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","query":"rdkafka_top_msg_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Size in Producer Queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of messages transmitted (produced) to Kafka brokers","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":255,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_tx_msgs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","query":"rdkafka_top_tx_msgs{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Produced Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of messages consumed, not including ignored messages (due to offset, etc), from Kafka brokers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":256,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_rx_msgs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","query":"rdkafka_top_rx_msgs{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Received Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages awaiting transmission to broker","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":257,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_outbuf_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","query":"rdkafka_broker_outbuf_msg_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Count Pending to Transmit (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages in-flight to broker awaiting response","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":258,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_waitresp_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","query":"rdkafka_broker_waitresp_msg_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Inflight Message Count (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of transmission errors","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":259,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_tx_errs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","query":"rdkafka_broker_tx_errs{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Error Count When Transmitting (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of receive errors","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":260,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rx_errs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","query":"rdkafka_broker_rx_errs{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Error Count When Receiving (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of requests timed out","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":261,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_req_timeouts{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","query":"rdkafka_broker_req_timeouts{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Timeout Request Count (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Broker latency / round-trip time in milli seconds","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":262,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_avg{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_avg{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p75{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_p75{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p90{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_p90{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_p99{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_rtt_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"RTT (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Broker throttling time in milliseconds","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":263,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_avg{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_avg{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p75{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_p75{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p90{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_p90{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_p99{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","query":"rdkafka_broker_throttle_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throttle Time (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Age of metadata from broker for this topic (milliseconds)","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":264,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_metadata_age{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}","metric":"","query":"rdkafka_topic_metadata_age{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Topic Metadata_age Age","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Batch sizes in bytes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":265,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_avg{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_avg{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p75{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_p75{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p90{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_p90{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_p99{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p99_99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_p99_99{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_out_of_range{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchsize_out_of_range{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Batch message counts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":null,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_avg{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_avg{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p75{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_p75{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p90{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_p90{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_p99{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p99_99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_p99_99{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_out_of_range{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","query":"rdkafka_topic_batchcnt_out_of_range{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Topic Batch Messages","transformations":[],"transparent":false,"type":"timeseries"}],"timeFrom":null,"timeShift":null,"title":"Topic Batch Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages ready to be produced in transmit queue","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":266,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_xmit_msgq_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","query":"rdkafka_topic_partition_xmit_msgq_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message to be Transmitted","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of pre-fetched messages in fetch queue","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":267,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_fetchq_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","query":"rdkafka_topic_partition_fetchq_cnt{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message in pre fetch queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Next offset to fetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":268,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_next_offset{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","query":"rdkafka_topic_partition_next_offset{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Next offset to fetch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Last committed offset","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":269,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_committed_offset{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","query":"rdkafka_topic_partition_committed_offset{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Committed Offset","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Kafka Native Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":270,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":271,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} read @ {{instance}}","metric":"","query":"sum(rate(connection_read_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} write @ {{instance}}","metric":"","query":"sum(rate(connection_write_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Network throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":272,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} read @ {{instance}}","metric":"","query":"sum(rate(connection_read_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} write @ {{instance}}","metric":"","query":"sum(rate(connection_write_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"S3 throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":273,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} read @ {{instance}}","metric":"","query":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} write @ {{instance}}","metric":"","query":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total read @ {{instance}}","metric":"","query":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total write @ {{instance}}","metric":"","query":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"gRPC throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":274,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_io_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","query":"sum(irate(connection_io_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_io_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} grpc {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","query":"sum(rate(connection_io_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_io_err_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","query":"sum(rate(connection_io_err_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"IO error rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":275,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(connection_count{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","query":"sum(connection_count{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(connection_count{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","query":"sum(connection_count{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}) by (job, instance, connection_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Existing connection count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":276,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_create_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","query":"sum(irate(connection_create_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_create_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","query":"sum(irate(connection_create_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create new connection rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":277,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","query":"sum(irate(connection_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","query":"sum(irate(connection_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create new connection err rate","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Network connection","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":27},"height":null,"hideTimeOverride":false,"id":278,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"iceberg file appender write qps","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":279,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"iceberg_file_appender_write_qps{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} @ {{sink_id}}","metric":"","query":"iceberg_file_appender_write_qps{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Qps Of Iceberg File Appender","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":280,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 @ {{sink_id}}","metric":"","query":"histogram_quantile(0.5, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 @ {{sink_id}}","metric":"","query":"histogram_quantile(0.99, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax @ {{sink_id}}","metric":"","query":"histogram_quantile(1.0, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, sink_id)(rate(iceberg_file_appender_write_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(iceberg_file_appender_write_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg @ {{sink_id}}","metric":"","query":"sum by(le, sink_id)(rate(iceberg_file_appender_write_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(iceberg_file_appender_write_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write latency Of Iceberg File Appender","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Iceberg Sink Metrics","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, instance)","description":"Reporting instance of the metric","hide":0,"includeAll":true,"label":"Node","multi":true,"name":"node","options":[],"query":{"query":"label_values(process_cpu_seconds_total, instance)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, job)","description":"Reporting job of the metric","hide":0,"includeAll":true,"label":"Job","multi":true,"name":"job","options":[],"query":{"query":"label_values(process_cpu_seconds_total, job)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(table_info, table_id)","description":"Reporting table id of the metric","hide":0,"includeAll":true,"label":"Table","multi":true,"name":"table","options":[],"query":{"query":"label_values(table_info, table_id)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"}]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dev_dashboard","uid":"Ecy3uV1nz","version":0} +{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dev Dashboard","editable":true,"gnetId":null,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from actor id to fragment id","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Id Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from materialized view table id to it's internal table ids","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"table_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Actor/Table Id Info","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of each type of RisingWave components alive.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":5,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The memory usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":6,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The CPU usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":7,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cpu usage (total) - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cpu usage (avg per core) - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"RW cluster can configure multiple meta nodes to achieve high availability. One is the leader and the rest are the followers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":8,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(meta_num{job=~\"$job\",instance=~\"$node\"}) by (worker_addr,role)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_addr}} @ {{role}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Meta Cluster","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Cluster Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":2},"height":null,"hideTimeOverride":false,"id":9,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The rate of successful recovery attempts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":10,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Recovery Successful Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of failed reocovery attempts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":11,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Failed recovery attempts","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Time spent in a successful recovery attempt","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":12,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(recovery_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency pmax - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by (le) (rate(recovery_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by (le) (rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"recovery latency avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Recovery latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Recovery","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":3},"height":null,"hideTimeOverride":false,"id":13,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":14,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Each query is executed in parallel with a user-defined parallelism. This figure shows the throughput of each parallelism. The throughput of all the parallelism added up is equal to Source Throughput(rows).","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":15,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(partition_input_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":16,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Each query is executed in parallel with a user-defined parallelism. This figure shows the throughput of each parallelism. The throughput of all the parallelism added up is equal to Source Throughput(MB/s).","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":17,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"RisingWave ingests barriers periodically to trigger computation and checkpoints. The frequency of barrier can be set by barrier_interval_ms. This metric shows how many rows are ingested between two consecutive barriers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":18,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_rows_per_barrier_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) per barrier","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Monitor each source upstream, 0 means the upstream is not normal, 1 means the source is ready.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":19,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"source_status_is_up{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source_id={{source_id}}, source_name={{source_name}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Upstream Status","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Source Split Change Events frequency by source_id and actor_id","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":20,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_split_change_event_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Split Change Events frequency(events/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Kafka Consumer Lag Size by source_id, partition and actor_id","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":21,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"high_watermark{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}} partition={{partition}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"latest_message_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}} partition={{partition}} actor_id={{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kafka Consumer Lag Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":22,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":23,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id, actor_id) * on(actor_id) group_left(sink_name) sink_info{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}} - actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s) per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":24,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":25,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (actor_id, table_id) * on(actor_id, table_id) group_left(table_name) table_info{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}} - actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s) per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been read from the backfill snapshot","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":26,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_snapshot_read_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Snapshot Read Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been output from the backfill upstream","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":27,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_upstream_output_row_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Upstream Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of barriers that have been ingested but not completely processed. This metric reflects the current level of congestion within the system.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":28,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all_barrier","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"in_flight_barrier_nums{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"in_flight_barrier","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The duration between the time point when the scheduled barrier needs to be sent and the time point when the barrier gets actually sent to all the compute nodes. Developers can thus detect any internal congestion.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":29,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_send_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Send Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The time that the data between two consecutive barriers gets fully processed, i.e. the computation results are made durable into materialized views or sink to external systems. This metric shows to users the freshness of materialized views.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":30,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":31,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier In-Flight Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":32,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p999 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_pmax - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_avg - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Sync Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":33,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_wait_commit_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Wait Commit Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of actors that have processed the earliest in-flight barriers per second. This metric helps users to detect potential congestion or stuck in the system.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":34,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_barrier_manager_progress{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Earliest In-Flight Barrier Progress","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":4},"height":null,"hideTimeOverride":false,"id":35,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"We first record the total blocking duration(ns) of output buffer of each actor. It shows how much time it takes an actor to process a message, i.e. a barrier, a watermark or rows of data, on average. Then we divide this duration by 1 second and show it as a percentage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":36,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}->{{downstream_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Blocking Time Ratio (Backpressure)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":37,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_input_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, upstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}<-{{upstream_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":38,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_in_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":39,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_out_record_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The operator-level memory usage statistics collected by each LRU cache","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":40,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (table_id, desc)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} desc: {{desc}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_memory_usage{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} actor {{actor_id}} desc: {{desc}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Memory Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Memory usage aggregated by materialized views","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":41,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Memory Usage of Materialized Views","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":42,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"temporal join cache miss, table_id {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"temporal join cache miss, table_id {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Temporal Join Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":43,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache hit count - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total cached count - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_hit_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache hit count - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_total_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total cached count - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialize Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":44,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache lookup count - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss count - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_over_window_cache_lookup_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache lookup count - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_over_window_cache_miss_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss count - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Over Window Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":45,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"join executor cache miss ratio - - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n appendonly cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream lookup cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream temporal join cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialize executor cache miss ratio - table {{table_id}} fragment {{fragment_id}} {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_over_window_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id) ) / (sum(rate(stream_over_window_cache_lookup_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Over window cache miss ratio - table {{table_id}} fragment {{fragment_id}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Miss Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":46,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, wait_side, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, fragment_id, wait_side, job)(rate(stream_join_barrier_align_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le,fragment_id,wait_side,job) (rate(stream_join_barrier_align_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - fragment {{fragment_id}} {{wait_side}} - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Barrier Align","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":47,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_actor_input_waiting_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":48,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}} {{side}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_match_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / 1000000000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Match Duration Per Second","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Multiple rows with distinct primary keys may have the same join key. This metric counts the number of join keys in the executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":49,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (fragment_id, side)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}} {{side}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of matched rows on the opposite side","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":50,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_matched_join_keys_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, fragment_id, table_id, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, actor_id, table_id) (rate(stream_join_matched_join_keys_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, fragment_id, table_id) (rate(stream_join_matched_join_keys_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - fragment {{fragment_id}} table_id {{table_id}} - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Matched Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":51,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups - table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss - table {{table_id}} actor {{actor_id}}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each StreamChunk","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":52,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg cached keys count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg distinct cached keys count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg cached keys count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_distinct_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg distinct cached keys count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of dirty (unflushed) groups in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":53,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_dirty_groups_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Dirty Groups Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The total heap size of dirty (unflushed) groups in each hash aggregation executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":54,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups heap size | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_dirty_groups_heap_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stream agg dirty groups heap size | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Dirty Groups Heap Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in each top_n executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":55,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n appendonly cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_group_top_n_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n cached count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_group_top_n_appendonly_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"group top_n appendonly cached count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"TopN Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in temporal join executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":56,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal Join cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_temporal_join_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal Join cached count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Temporal Join Cache Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in lookup executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":57,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lookup cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_lookup_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lookup cached count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lookup Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of keys cached in over window executor's executor cache.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":58,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}) by (table_id, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"over window cached count | table {{table_id}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_over_window_cached_entry_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"over window cached count | table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Over Window Cached Keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"When enabled, this metric shows the input throughput of each executor.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":59,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_identity, fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_identity}} fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_executor_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_identity}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The actor-level memory usage statistics reported by TaskLocalAlloc. (Disabled by default)","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":60,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(actor_memory_usage{job=~\"$job\",instance=~\"$node\"}) by (fragment_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_memory_usage{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Memory Usage (TaskLocalAlloc)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":5},"height":null,"hideTimeOverride":false,"id":61,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":62,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_actor_execution_time{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Execution Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":63,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":64,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":8},"height":null,"hideTimeOverride":false,"id":65,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_fast_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":8},"height":null,"hideTimeOverride":false,"id":66,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":67,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":16},"height":null,"hideTimeOverride":false,"id":68,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_slow_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":16},"height":null,"hideTimeOverride":false,"id":69,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":70,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":24},"height":null,"hideTimeOverride":false,"id":71,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_poll_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":24},"height":null,"hideTimeOverride":false,"id":72,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":73,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":32},"height":null,"hideTimeOverride":false,"id":74,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_idle_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":32},"height":null,"hideTimeOverride":false,"id":75,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":76,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":40},"height":null,"hideTimeOverride":false,"id":77,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(stream_actor_scheduled_cnt{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Avg Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors (Tokio)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":6},"height":null,"hideTimeOverride":false,"id":78,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":79,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Send Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":80,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Recv Throughput","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Exchange","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":7},"height":null,"hideTimeOverride":false,"id":81,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":82,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compute Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":83,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":84,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_reader_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: actor_id={{actor_id}}, source_id={{source_id}})","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_reader_error{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, actor_id, source_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: actor_id={{actor_id}}, source_id={{source_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Reader Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":85,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_sink_error{job=~\"$job\",instance=~\"$node\"}) by (connector_name, executor_id, error_msg)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{connector_name}}: {{error_msg}} ({{executor_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink by Connector","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"User Streaming Errors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":86,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":87,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Exchange Recv Row Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":88,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mpp Task Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"All memory usage of batch executors in bytes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":89,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_total_mem{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mem Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":90,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_heartbeat_worker_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Heartbeat Worker Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the memory usage of mem_table.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":91,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table size total - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_mem_table_memory_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table size - table id {{table_id}} instance id {{instance_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Mem Table Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the item counts in mem_table.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":92,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table counts total - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_mem_table_item_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem_table count - table id {{table_id}} instance id {{instance_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Mem Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":93,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Row SeqScan Next Duration","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":94,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":95,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{table_id}} @ {{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total_meta_miss_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Hummock has three parts of memory usage: 1. Meta Cache 2. Block CacheThis metric shows the real memory usage of each of these three caches.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":96,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"data cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":97,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"block cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Miss Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":98,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_scan_key_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type, table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{table_id}} @ {{type}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Iter keys flow","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":99,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p50 - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p99 - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts pmax - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Merged SSTs","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of the latency of Get operations that have been issued to the state store.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":100,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of the time spent on iterator initialization.Histogram of the time spent on iterator scanning.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":101,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_init_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_init_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_init_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_scan_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_iter_scan_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":102,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive count - {{table_id}} - {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter positive count - {{table_id}} - {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter check count- {{table_id}} - {{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Positive / Total","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":103,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_read_req_bloom_filter_positive_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter positive rate - {{table_id}} - {{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter Positive Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"False-Positive / Total","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":104,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(((sum(rate(state_store_read_req_positive_but_non_exist_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter false positive rate - {{table_id}} - {{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Bloom Filter False-Positive Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":105,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_iter_slow_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Slow Fetch Meta Unhits","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":106,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_shared_buffer_hit_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer hit - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_in_process_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":107,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":108,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":109,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.5, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.5, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.5, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read p50 - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read p99 - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id) + sum((histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read pmax - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Read Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":110,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Count - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size of a single key-value pair when reading by operation Get.Operation Get gets a single key-value pair with respect to a caller-specified key. If the key does not exist in the storage, the size of key is counted into this metric and the size of value is 0.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":111,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size of all the key-value paris when reading by operation Iter.Operation Iter scans a range of key-value pairs.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":112,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":113,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":114,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_iter_fetch_meta_cache_unhits{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Unhits","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock (Read)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":10},"height":null,"hideTimeOverride":false,"id":115,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the real memory usage of uploader.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":116,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading memory - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_uploader_uploading_task_size{job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading task size - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader Memory Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Histogram of time spent on compacting shared buffer to remote storage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":117,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Build and Sync Sstable Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":118,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.5, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write p50 - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(0.99, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write p99 - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id)) * on(table_id) group_left(materialized_view_id) (group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write pmax - materialized view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Write Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":119,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_merge_imm_task_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"merge imm tasks - {{table_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_spill_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Uploader spill tasks - {{uploader_stage}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader - Tasks Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":120,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_merge_imm_memory_sz{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Merging tasks memory size - {{table_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_spill_task_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,uploader_stage)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Uploading tasks size - {{uploader_stage}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Uploader - Task Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":121,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write batch - {{table_id}} @ {{job}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sync_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"l0 - {{job}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":122,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":123,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_write_batch_tuple_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write_batch_kv_pair_count - {{table_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Item Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":124,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_size_sum{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id) / sum(rate(state_store_write_batch_size_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_shared_buffer_to_sstable_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance) / sum(rate(compactor_shared_buffer_to_sstable_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sync - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric shows the statistics of mem_table size on flush. By default only max (p100) is shown.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":125,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, table_id, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_write_batch_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, table_id, job, instance) (rate(state_store_write_batch_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Batch Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":126,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_mem_table_spill_counts{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mem table spill table id - {{table_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Mem Table Spill Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":127,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Sync Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock (Write)","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":11},"height":null,"hideTimeOverride":false,"id":128,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of SSTables at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":129,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SSTable Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The size(KB) of SSTables at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":130,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SSTable Size(KB)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The of bytes that have been written by commit epoch per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":131,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_commit_write_throughput{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{table_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Commit Flush Bytes by Table","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have completed or failed","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":132,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency{result!='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Failure Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have completed or failed","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":133,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency{result='SUCCESS',job=~\"$job\",instance=~\"$node\"}) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Success Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that have been skipped.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":134,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_skip_compact_frequency{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (level, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{level}}-{{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Skip Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg l0 select_level_count of the compact task, and categorize it according to different cg, levels and task types","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":135,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, group, type)(irate(storage_l0_compact_level_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_l0_compact_level_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg cg{{group}}@{{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task L0 Select Level Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg file count of the compact task, and categorize it according to different cg, levels and task types","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":136,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, group, type)(irate(storage_compact_task_file_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, group, type)(irate(storage_compact_task_file_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg cg{{group}}@{{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task File Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The distribution of the compact task size triggered, including p90 and max. and categorize it according to different cg, levels and task types.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":137,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - cg{{group}}@{{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(storage_compact_task_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - cg{{group}}@{{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Task Size Distribution","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of compactions from one level to another level that are running.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":138,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(storage_compact_task_pending_num{job=~\"$job\",instance=~\"$node\"}) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor_task_split_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Running Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"compact-task: The total time have been spent on compaction.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":139,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p50 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p90 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compact_task_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task pmax - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range p90 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compact_sst_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range pmax - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id p90 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id pmax - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(compute_refill_cache_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compute_apply_version_duration_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(compactor_compact_task_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(state_store_compact_sst_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"KBs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":140,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job) + sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_fast_compact_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fast compact - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of bytes that have been written by compaction.Flush refers to the process of compacting Memtables to SSTables at Level 0.Write refers to the process of compacting SSTables at one level to another level.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":141,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Bytes(GiB)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Write amplification is the amount of bytes written to the remote storage by compaction for each one byte of flushed SSTable data. Write amplification is by definition higher than 1.0 because we write each piece of data to L0, and then write it again to an SSTable, and then compaction may read this piece of data and write it to a new SSTable, that's another write.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":142,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) / sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"})","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write amplification","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Amplification","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of SSTables that is being compacted at each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":143,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting SSTable Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"num of compact_task","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":144,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_task_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":145,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from next level","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from current level","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} write to next level","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read/Write by Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":146,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_write_sstn{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} write to next level","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_read_sstn_next{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from next level","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_level_compact_read_sstn_curr{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, group, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cg{{group}}-L{{level_index}} read from current level","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read/Write by level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total bytes gotten from sstable_bloom_filter, for observing bloom_filter size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":147,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_meta - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_file - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total bytes gotten from sstable_avg_key_size, for observing sstable_avg_key_size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":148,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_key_size - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_value_size - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Item Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Avg count gotten from sstable_distinct_epoch_count, for observing sstable_distinct_epoch_count","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":149,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_epoch_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Stat","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total time of operations which read from remote storage when enable prefetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":150,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Remote Read Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":151,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_iter_scan_key_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{type}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Iter keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"bytes of Lsm tree needed to reach balance","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":152,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_pending_bytes{job=~\"$job\",instance=~\"$node\"}) by (instance, group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact pending bytes - {{group}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Compact Pending Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"compression ratio of each level of the lsm tree","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":153,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_level_compression_ratio{job=~\"$job\",instance=~\"$node\"}) by (instance, group, level, algorithm)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lsm compression ratio - cg{{group}} @ L{{level}} - {{algorithm}} {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Level Compression Ratio","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Compaction","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":12},"height":null,"hideTimeOverride":false,"id":154,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":155,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":156,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum{type!~'streaming_upload_write_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":157,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type!~'streaming_upload_write_bytes|streaming_read_read_bytes|streaming_read',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'upload|delete',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-write - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-read - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":158,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":159,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Failure Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":160,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(aws_sdk_retry_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(s3_read_request_retry_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Retry Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"There are two types of operations: 1. GET, SELECT, and DELETE, they cost 0.0004 USD per 1000 requests. 2. PUT, COPY, POST, LIST, they cost 0.005 USD per 1000 requests.Reading from S3 across different regions impose extra cost. This metric assumes 0.01 USD per 1GB data transfer. Please checkout AWS's pricing model for more accurate calculation.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":161,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}) * 0.01 / 1000 / 1000 / 1000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"(Cross Region) Data Transfer Cost","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete',job=~\"$job\",instance=~\"$node\"}) * 0.0004 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GET, SELECT, and all other Requests Cost","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list',job=~\"$job\",instance=~\"$node\"}) * 0.005 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"PUT, COPY, POST, LIST Requests Cost","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Realtime)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"This metric uses the total size of data in S3 at this second to derive the cost of storing data for a whole month. The price is 0.023 USD per GB. Please checkout AWS's pricing model for more accurate calculation.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":162,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size{job=~\"$job\",instance=~\"$node\"}) by (instance) * 0.023 / 1000 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Monthly Storage Cost","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Monthly)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Object Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":13},"height":null,"hideTimeOverride":false,"id":163,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":164,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache {{op}} {{extra}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":165,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(foyer_storage_op_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, foyer, op, extra, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":166,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, op, extra, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache - {{op}} {{extra}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":167,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(foyer_storage_total_bytes{job=~\"$job\",instance=~\"$node\"}) by (foyer, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} size @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":168,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) / (sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"hit\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance) + sum(rate(foyer_storage_op_duration_count{op=\"lookup\",extra=\"miss\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (foyer, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{foyer}} file cache hit ratio @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Hit Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":169,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(refill_queue_total) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"refill queue length @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Queue Length","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":170,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(refill_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} file cache refill - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(refill_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (type, op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} file cache refill - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":171,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(refill_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, type, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{type}} file cache refill - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Refill Latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Tiered Cache","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":14},"height":null,"hideTimeOverride":false,"id":172,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":173,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p50 - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p99 - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time pmax - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lock Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":174,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p50 - {{method}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p99 - {{method}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time pmax - {{method}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Real Process Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":175,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version size","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":176,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"current version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"checkpoint version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_safepoint_version_id{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min safepoint version id","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Id","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":177,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"max committed epoch","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"safe epoch","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned epoch","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":178,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_value_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":179,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_materialized_view_stats{metric='materialized_view_total_size',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{metric}}, mv id - {{table_id}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":180,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_count',table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\nObjects are classified into 3 groups:\n- not referenced by versions: these object are being deleted from object store.\n- referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n- referenced by current version: these objects are in the latest version.\n\nAdditionally, a metric on all objects (including dangling ones) is updated with low-frequency. The metric is updated right before full GC. So subsequent full GC may reduce the actual value significantly, without updating the metric.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":181,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_total_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all objects (including dangling ones)","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Refer to `Object Total Number` panel for classification of objects.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":182,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_total_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all objects, including dangling ones","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"total number of hummock version delta log","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":183,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_delta_log_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"delta log total number","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Delta Log Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"hummock version checkpoint latency","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":184,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(storage_version_checkpoint_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(storage_version_checkpoint_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(storage_version_checkpoint_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version_checkpoint_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Checkpoint Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"When certain per compaction group threshold is exceeded (e.g. number of level 0 sub-level in LSMtree), write op to that compaction group is stopped temporarily. Check log for detail reason of write stop.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":185,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_write_stop_compaction_groups{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compaction_group_{{compaction_group_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Stop Compaction Groups","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"total number of attempts to trigger full GC","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":186,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_full_gc_trigger_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"full_gc_trigger_count","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Full GC Trigger Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"the object id watermark used in last full GC","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":187,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_full_gc_last_object_id_watermark{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"full_gc_last_object_id_watermark","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Full GC Last Watermark","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":188,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(storage_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta consumed latency pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(storage_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta iteration latency pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_consumed_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor consumed latency pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(irate(compactor_compaction_event_loop_iteration_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor iteration latency pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Event Loop Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The times of move_state_table occurs","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":189,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_move_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"move table cg{{group}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Move State Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of state_tables in each CG","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":190,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_state_table_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"state table cg{{group}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"State Table Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of branched_sst in each CG","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":191,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(storage_branched_sst_count{table_id=~\"$table|\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"branched sst cg{{group}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Branched SST Count","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":15},"height":null,"hideTimeOverride":false,"id":192,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total backup job count since the Meta node starts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":193,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"backup_job_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"job count","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Latency of backup jobs since the Meta node starts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":194,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p50 - {{state}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p99 - {{state}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time pmax - {{state}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Process Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Backup Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":195,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":196,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":197,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Drop latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":198,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetCatalog latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Catalog Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":17},"height":null,"hideTimeOverride":false,"id":199,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":200,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"AddWorkerNode latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":201,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ListAllNodes latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Cluster Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":202,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":203,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"CreateMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":204,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"DropMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":205,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Flush latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Stream Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":19},"height":null,"hideTimeOverride":false,"id":206,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":207,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinVersionBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":208,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinSnapshotBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":209,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ReportCompactionTasks latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":210,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds',job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetNewSstIds latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":20},"height":null,"hideTimeOverride":false,"id":211,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":212,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_counts - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":213,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_version_before_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"version_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":214,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latencyp90 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":215,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_counts - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_counts - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":216,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":217,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_counts - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":218,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC: Hummock Meta Client","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":21},"height":null,"hideTimeOverride":false,"id":219,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of active sessions","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":220,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Active Sessions","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":221,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per Second (Local Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":222,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per Second (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":223,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Running Queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":224,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Rejected queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":225,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The Number of Completed Queries (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":226,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency (Distributed Query Mode)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":227,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency (Local Query Mode)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Frontend","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":22},"height":null,"hideTimeOverride":false,"id":228,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":229,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(lru_runtime_loop_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager loop count per sec","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":230,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_watermark_step{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark steps","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"watermark_time is the current lower watermark of cached data. physical_now is the current time of the machine. The diff (physical_now - watermark_time) shows how much data is cached.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":231,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_physical_now_ms{job=~\"$job\",instance=~\"$node\"} - lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between watermark_time and now (ms)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":232,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The allocated memory of jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":233,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_active_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The active memory of jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":234,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jvm_allocated_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The allocated memory of jvm","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":235,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jvm_active_bytes{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The active memory of jvm","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":236,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_current_watermark_time_ms{job=~\"$job\",instance=~\"$node\"} - on() group_right() lru_evicted_watermark_time_ms{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table {{table_id}} actor {{actor_id}} desc: {{desc}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between current watermark and evicted watermark time (ms) for actors","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":23},"height":null,"hideTimeOverride":false,"id":237,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":238,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_type}} @ {{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":239,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector_type}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Sink Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Connector Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":240,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":241,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 @ {{connector}} {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 @ {{connector}} {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(sink_commit_duration_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, connector, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax @ {{connector}} {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, connector, sink_id)(rate(sink_commit_duration_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(sink_commit_duration_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{connector}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Commit Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":242,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"latest write epoch @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"latest read epoch @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Read/Write Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":243,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(max(log_store_latest_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Consume lag @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Lag","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":244,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"clamp_min((max(log_store_first_write_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)- max(log_store_latest_read_epoch{job=~\"$job\",instance=~\"$node\"}) by (connector, sink_id, executor_id)) / (2^16) / 1000, 0)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Consume persistent log lag @ {{connector}} {{sink_id}} {{executor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Consume Persistent Log Lag","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":245,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Consume Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":246,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_read_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}} @ {{executor_id}} {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Log Store Consume Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":247,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Log Store Write Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":248,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(log_store_write_rows{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, connector, sink_id, executor_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector}} {{sink_id}} @ {{executor_id}} {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Log Store Write Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":249,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_read_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Read Storage Row Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":250,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_read_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Read Storage Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":251,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_write_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Write Storage Row Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":252,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(kv_log_store_storage_write_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (executor_id, connector, sink_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} - {{connector}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Kv Log Store Write Storage Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Sink Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":25},"height":null,"hideTimeOverride":false,"id":253,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Current number of messages in producer queues","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":254,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Count in Producer Queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Current total size of messages in producer queues","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":255,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_msg_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Size in Producer Queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of messages transmitted (produced) to Kafka brokers","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":256,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_tx_msgs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Produced Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of messages consumed, not including ignored messages (due to offset, etc), from Kafka brokers.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":257,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_top_rx_msgs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Received Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages awaiting transmission to broker","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":258,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_outbuf_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message Count Pending to Transmit (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages in-flight to broker awaiting response","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":259,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_waitresp_msg_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Inflight Message Count (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of transmission errors","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":260,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_tx_errs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Error Count When Transmitting (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of receive errors","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":261,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rx_errs{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Error Count When Receiving (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of requests timed out","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":262,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_req_timeouts{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, state {{ state }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Timeout Request Count (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Broker latency / round-trip time in milli seconds","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":263,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_avg{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p75{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p90{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_rtt_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"RTT (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Broker throttling time in milliseconds","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":264,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_avg{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p75{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p90{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_p99_99{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_broker_throttle_out_of_range{job=~\"$job\",instance=~\"$node\"}/1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throttle Time (per broker)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Age of metadata from broker for this topic (milliseconds)","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ms"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":265,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_metadata_age{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Topic Metadata_age Age","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Batch sizes in bytes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":266,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_avg{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p75{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p90{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_p99_99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchsize_out_of_range{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Batch message counts","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":null,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_avg{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p75{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p90{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_p99_99{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_batchcnt_out_of_range{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, broker {{ broker }}, topic {{ topic }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Topic Batch Messages","transformations":[],"transparent":false,"type":"timeseries"}],"timeFrom":null,"timeShift":null,"title":"Topic Batch Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of messages ready to be produced in transmit queue","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":267,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_xmit_msgq_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message to be Transmitted","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of pre-fetched messages in fetch queue","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":268,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_fetchq_cnt{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Message in pre fetch queue","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Next offset to fetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":269,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_next_offset{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Next offset to fetch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Last committed offset","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":270,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rdkafka_topic_partition_committed_offset{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"id {{ id }}, client_id {{ client_id}}, topic {{ topic }}, partition {{ partition }}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Committed Offset","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Kafka Native Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":271,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":272,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} read @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} write @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Network throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":273,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} read @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} write @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"S3 throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":274,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} read @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} write @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_read_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total read @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_write_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance) / (1024*1024)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total write @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"gRPC throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":275,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_io_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_io_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} grpc {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(connection_io_err_rate{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, op_type, error_kind)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} total {{op_type}} err[{{error_kind}}] @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"IO error rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":276,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(connection_count{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(connection_count{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Existing connection count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":277,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_create_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_create_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create new connection rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":278,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_err_rate{connection_type=\"S3\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} S3 @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(connection_err_rate{connection_type=~\"grpc.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, connection_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} {{connection_type}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create new connection err rate","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Network connection","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":27},"height":null,"hideTimeOverride":false,"id":279,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"iceberg file appender write qps","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":280,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"iceberg_file_appender_write_qps{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{executor_id}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Qps Of Iceberg File Appender","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":281,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 @ {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 @ {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(iceberg_file_appender_write_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, sink_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax @ {{sink_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, sink_id)(rate(iceberg_file_appender_write_latency_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, type, job, instance) (rate(iceberg_file_appender_write_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write latency Of Iceberg File Appender","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Iceberg Sink Metrics","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, instance)","description":"Reporting instance of the metric","hide":0,"includeAll":true,"label":"Node","multi":true,"name":"node","options":[],"query":{"query":"label_values(process_cpu_seconds_total, instance)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, job)","description":"Reporting job of the metric","hide":0,"includeAll":true,"label":"Job","multi":true,"name":"job","options":[],"query":{"query":"label_values(process_cpu_seconds_total, job)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(table_info, table_id)","description":"Reporting table id of the metric","hide":0,"includeAll":true,"label":"Table","multi":true,"name":"table","options":[],"query":{"query":"label_values(table_info, table_id)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"}]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dev_dashboard","uid":"Ecy3uV1nz","version":0} diff --git a/grafana/risingwave-user-dashboard.json b/grafana/risingwave-user-dashboard.json index 63e3ed095ac69..41445ec1e30fb 100644 --- a/grafana/risingwave-user-dashboard.json +++ b/grafana/risingwave-user-dashboard.json @@ -1 +1 @@ -{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dashboard","editable":true,"gnetId":null,"graphTooltip":0,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Actor/Table Id Info","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from actor id to fragment id","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"actor_info{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Id Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from materialized view table id to it's internal table ids","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":12,"y":1},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id, table_name, table_type)","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id, table_name, table_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Overview","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":10},"height":null,"hideTimeOverride":false,"id":5,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (source_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{source_name}}","metric":"","query":"sum(rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (source_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregated Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":10},"height":null,"hideTimeOverride":false,"id":6,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source_id {{source_id}}","metric":"","query":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregated Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":7,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}}","metric":"","query":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":18},"height":null,"hideTimeOverride":false,"id":8,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}}","metric":"","query":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The time that the data between two consecutive barriers gets fully processed, i.e. the computation results are made durable into materialized views or sink to external systems. This metric shows to users the freshness of materialized views.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":9,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","query":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Alerts in the system group by type:\n - Too Many Barriers: there are too many uncommitted barriers generated. This means the streaming graph is stuck or under heavy load. Check 'Barrier Latency' panel.\n - Recovery Triggered: cluster recovery is triggered. Check 'Errors by Type' / 'Node Count' panels.\n - Lagging Version: the checkpointed or pinned version id is lagging behind the current version id. Check 'Hummock Manager' section in dev dashboard.\n - Lagging Epoch: the pinned or safe epoch is lagging behind the current max committed epoch. Check 'Hummock Manager' section in dev dashboard.\n - Lagging Compaction: there are too many files in L0. This can be caused by compactor failure or lag of compactor resource. Check 'Compaction' section in dev dashboard.\n - Lagging Vacuum: there are too many stale files waiting to be cleaned. This can be caused by compactor failure or lag of compactor resource. Check 'Compaction' section in dev dashboard.\n - Abnormal Meta Cache Memory: the meta cache memory usage is too large, exceeding the expected 10 percent.\n - Abnormal Block Cache Memory: the block cache memory usage is too large, exceeding the expected 10 percent.\n - Abnormal Uploading Memory Usage: uploading memory is more than 70 percent of the expected, and is about to spill.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":26},"height":null,"hideTimeOverride":false,"id":10,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"} >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Too Many Barriers","metric":"","query":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"} >= bool 200","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) > bool 0 + sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) > bool 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Recovery Triggered","metric":"","query":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) > bool 0 + sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) > bool 0","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100) + ((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Version","metric":"","query":"((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100) + ((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"} == 0) + ((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_safe_epoch{job=~\"$job\",instance=~\"$node\"} == 0)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Epoch","metric":"","query":"((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"} == 0) + ((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_safe_epoch{job=~\"$job\",instance=~\"$node\"} == 0)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(label_replace(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}, 'L0', 'L0', 'level_index', '.*_L0') unless storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (L0) >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Compaction","metric":"","query":"sum(label_replace(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}, 'L0', 'L0', 'level_index', '.*_L0') unless storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (L0) >= bool 200","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"} >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Vacuum","metric":"","query":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"} >= bool 200","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_meta_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Meta Cache Memory","metric":"","query":"state_store_meta_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_block_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Block Cache Memory","metric":"","query":"state_store_block_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_uploading_memory_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 0.7","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Uploading Memory Usage","metric":"","query":"state_store_uploading_memory_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 0.7","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Alerts","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Errors in the system group by type","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":34},"height":null,"hideTimeOverride":false,"id":11,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compute error {{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","query":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"parse error {{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","query":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"source_status_is_up{job=~\"$job\",instance=~\"$node\"} == 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source error: source_id={{source_id}}, source_name={{source_name}} @ {{instance}}","metric":"","query":"source_status_is_up{job=~\"$job\",instance=~\"$node\"} == 0","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote storage error {{type}}: {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Errors","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":34},"height":null,"hideTimeOverride":false,"id":12,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Local mode","metric":"","query":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distributed mode","metric":"","query":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Query QPS","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of each type of RisingWave components alive.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":42},"height":null,"hideTimeOverride":false,"id":13,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","query":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of active sessions in frontend nodes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":42},"height":null,"hideTimeOverride":false,"id":14,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Active Sessions","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":50},"height":null,"hideTimeOverride":false,"id":15,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The CPU usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":16,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","query":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of CPU cores per RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":17,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","query":"avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU Core Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"CPU","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":51},"height":null,"hideTimeOverride":false,"id":18,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The memory usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":19,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":20,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage @ {{instance}}","metric":"","query":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Memory Usage (Total)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":21,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(actor_memory_usage[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"streaming actor - {{actor_id}}","metric":"","query":"rate(actor_memory_usage[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage meta cache - {{job}} @ {{instance}}","metric":"","query":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage block cache - {{job}} @ {{instance}}","metric":"","query":"sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage write buffer - {{job}} @ {{instance}}","metric":"","query":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized_view {{materialized_view_id}}","metric":"","query":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info) by (materialized_view_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Memory Usage (Detailed)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Executor cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":22,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Join - cache miss - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Join - total lookups - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n appendonly - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n appendonly - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lookup executor - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lookup executor - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal join - cache miss - table_id {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal join - total lookups - table_id {{table_id}} actor {{actor_id}}","metric":"","query":"rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Materialize - cache hit count - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","query":"rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Materialize - total cache count - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","query":"rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":23,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"join executor cache miss ratio - - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","query":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n appendonly cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream lookup cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream temporal join cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","query":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialize executor cache miss ratio - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","query":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Miss Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":24,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"memory cache - {{table_id}} @ {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_sst_store_block_request_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total_meta_miss_count - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage bloom filter statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":25,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_check_bloom_filter_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter total - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_check_bloom_filter_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_positive_but_non_exist_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_positive_but_non_exist_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Bloom Filer","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage file cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":26,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","query":"sum(rate(file_cache_miss{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage File Cache","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":52},"height":null,"hideTimeOverride":false,"id":27,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Send/Recv throughput per node for streaming exchange","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":28,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Send @ {{instance}}","metric":"","query":"sum(rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Recv @ {{instance}}","metric":"","query":"sum(rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Streming Remote Exchange (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The remote storage read/write throughput per node","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":29,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{instance}}","metric":"","query":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{instance}}","metric":"","query":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Remote I/O (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":30,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","query":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Exchange Recv (Rows/s)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Network","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":53},"height":null,"hideTimeOverride":false,"id":31,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\n Objects are classified into 3 groups:\n - not referenced by versions: these object are being deleted from object store.\n - referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n - referenced by current version: these objects are in the latest version.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":32,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","query":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","query":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","query":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The storage size of each materialized view","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":33,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_materialized_view_stats{metric='materialized_view_total_size',job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{metric}}, mv id - {{table_id}} ","metric":"","query":"storage_materialized_view_stats{metric='materialized_view_total_size',job=~\"$job\",instance=~\"$node\"}/1024","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\n Objects are classified into 3 groups:\n - not referenced by versions: these object are being deleted from object store.\n - referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n - referenced by current version: these objects are in the latest version.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":34,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","query":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","query":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","query":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of bytes that have been written by compaction.Flush refers to the process of compacting Memtables to SSTables at Level 0.Compaction refers to the process of compacting SSTables at one level to another level.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":35,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Compaction - {{job}}","metric":"","query":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush - {{job}}","metric":"","query":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The remote storage read/write throughput","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":36,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}}","metric":"","query":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","query":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Remote I/O (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Size statistics for checkpoint","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":37,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}}","metric":"","query":"sum by(le, job) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":54},"height":null,"hideTimeOverride":false,"id":38,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":39,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":40,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","query":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized executor actor per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":41,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_executor_row_count{executor_identity=~\".*MaterializeExecutor.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) * on(actor_id) group_left(materialized_view_id, table_name) (group(table_info{table_type=~\"MATERIALIZED_VIEW\",job=~\"$job\",instance=~\"$node\"}) by (actor_id, materialized_view_id, table_name))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized view {{table_name}} table_id {{materialized_view_id}}","metric":"","query":"sum(rate(stream_executor_row_count{executor_identity=~\".*MaterializeExecutor.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) * on(actor_id) group_left(materialized_view_id, table_name) (group(table_info{table_type=~\"MATERIALIZED_VIEW\",job=~\"$job\",instance=~\"$node\"}) by (actor_id, materialized_view_id, table_name))) by (materialized_view_id, table_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been read from the backfill operator used by MV on MV","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":42,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_snapshot_read_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Read Snapshot - table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_backfill_snapshot_read_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_upstream_output_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Upstream - table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","query":"rate(stream_backfill_upstream_output_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"We first record the total blocking duration(ns) of output buffer of each actor. It shows how much time it takes an actor to process a message, i.e. a barrier, a watermark or rows of data, on average. Then we divide this duration by 1 second and show it as a percentage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":43,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}->{{downstream_fragment_id}}","metric":"","query":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Blocking Time Ratio (Backpressure)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":55},"height":null,"hideTimeOverride":false,"id":44,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":45,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","query":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Running query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":46,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","query":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Rejected query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":47,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","query":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Completed query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":48,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":49,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Local Execution Mode","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":50,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":51,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_type}} @ {{source_id}}","metric":"","query":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":52,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector_type}} @ {{sink_id}}","metric":"","query":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Sink Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Connector Node","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, instance)","description":"Reporting instance of the metric","hide":0,"includeAll":true,"label":"Node","multi":true,"name":"node","options":[],"query":{"query":"label_values(process_cpu_seconds_total, instance)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, job)","description":"Reporting job of the metric","hide":0,"includeAll":true,"label":"Job","multi":true,"name":"job","options":[],"query":{"query":"label_values(process_cpu_seconds_total, job)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"}]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dashboard","uid":"Fcy3uV1nz","version":0} +{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dashboard","editable":true,"gnetId":null,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Actor/Table Id Info","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from actor id to fragment id","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"actor_info{job=~\"$job\",instance=~\"$node\"}","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Id Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"color":{"mode":"thresholds"},"columns":[],"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Mapping from materialized view table id to it's internal table ids","editable":true,"error":false,"fieldConfig":{"defaults":{"custom":{"align":"auto","displayMode":"auto","filterable":true},"thresholds":{"mode":"absolute","steps":[]}},"overrides":[]},"fontSize":"100%","gridPos":{"h":8,"w":12,"x":12,"y":1},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"mappings":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"showHeader":true},"repeat":null,"repeatDirection":null,"span":6,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"group(table_info{job=~\"$job\",instance=~\"$node\"}) by (materialized_view_id, table_id, table_name, table_type)","format":"table","hide":false,"instant":true,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Info","transformations":[{"id":"organize","options":{"excludeByName":{"Time":true,"Value":true,"__name__":true,"instance":true,"job":true}}}],"transparent":false,"type":"table"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Overview","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":10},"height":null,"hideTimeOverride":false,"id":5,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (source_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{source_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregated Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":10},"height":null,"hideTimeOverride":false,"id":6,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source_id {{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregated Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of rows streamed into each sink per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":7,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_sink_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group(sink_info{job=~\"$job\",instance=~\"$node\"}) by (sink_id, sink_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink {{sink_id}} {{sink_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Sink Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized view per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":18},"height":null,"hideTimeOverride":false,"id":8,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_mview_input_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id) * on(table_id) group_left(table_name) group(table_info{job=~\"$job\",instance=~\"$node\"}) by (table_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"mview {{table_id}} {{table_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The time that the data between two consecutive barriers gets fully processed, i.e. the computation results are made durable into materialized views or sink to external systems. This metric shows to users the freshness of materialized views.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":9,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) / rate(meta_barrier_duration_seconds_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Alerts in the system group by type:\n - Too Many Barriers: there are too many uncommitted barriers generated. This means the streaming graph is stuck or under heavy load. Check 'Barrier Latency' panel.\n - Recovery Triggered: cluster recovery is triggered. Check 'Errors by Type' / 'Node Count' panels.\n - Lagging Version: the checkpointed or pinned version id is lagging behind the current version id. Check 'Hummock Manager' section in dev dashboard.\n - Lagging Epoch: the pinned or safe epoch is lagging behind the current max committed epoch. Check 'Hummock Manager' section in dev dashboard.\n - Lagging Compaction: there are too many files in L0. This can be caused by compactor failure or lag of compactor resource. Check 'Compaction' section in dev dashboard.\n - Lagging Vacuum: there are too many stale files waiting to be cleaned. This can be caused by compactor failure or lag of compactor resource. Check 'Compaction' section in dev dashboard.\n - Abnormal Meta Cache Memory: the meta cache memory usage is too large, exceeding the expected 10 percent.\n - Abnormal Block Cache Memory: the block cache memory usage is too large, exceeding the expected 10 percent.\n - Abnormal Uploading Memory Usage: uploading memory is more than 70 percent of the expected, and is about to spill.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":26},"height":null,"hideTimeOverride":false,"id":10,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums{job=~\"$job\",instance=~\"$node\"} >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Too Many Barriers","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(recovery_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) > bool 0 + sum(recovery_failure_cnt{job=~\"$job\",instance=~\"$node\"}) > bool 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Recovery Triggered","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_checkpoint_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100) + ((storage_current_version_id{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_version_id{job=~\"$job\",instance=~\"$node\"}) >= bool 100)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Version","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_min_pinned_epoch{job=~\"$job\",instance=~\"$node\"} == 0) + ((storage_max_committed_epoch{job=~\"$job\",instance=~\"$node\"} - storage_safe_epoch{job=~\"$job\",instance=~\"$node\"}) >= bool 6553600000 unless + storage_safe_epoch{job=~\"$job\",instance=~\"$node\"} == 0)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Epoch","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(label_replace(storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}, 'L0', 'L0', 'level_index', '.*_L0') unless storage_level_sst_num{job=~\"$job\",instance=~\"$node\"}) by (L0) >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Compaction","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"} >= bool 200","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lagging Vacuum","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_meta_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Meta Cache Memory","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_block_cache_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 1.1","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Block Cache Memory","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"state_store_uploading_memory_usage_ratio{job=~\"$job\",instance=~\"$node\"} >= bool 0.7","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Abnormal Uploading Memory Usage","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Alerts","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Errors in the system group by type","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":34},"height":null,"hideTimeOverride":false,"id":11,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compute error {{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count{job=~\"$job\",instance=~\"$node\"}) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"parse error {{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"source_status_is_up{job=~\"$job\",instance=~\"$node\"} == 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source error: source_id={{source_id}}, source_name={{source_name}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote storage error {{type}}: {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Errors","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":34},"height":null,"hideTimeOverride":false,"id":12,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Local mode","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distributed mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Query QPS","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of each type of RisingWave components alive.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":42},"height":null,"hideTimeOverride":false,"id":13,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num{job=~\"$job\",instance=~\"$node\"}) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of active sessions in frontend nodes","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":42},"height":null,"hideTimeOverride":false,"id":14,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"frontend_active_sessions{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Active Sessions","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":50},"height":null,"hideTimeOverride":false,"id":15,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The CPU usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":16,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Number of CPU cores per RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":17,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_cpu_core_num{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU Core Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"CPU","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":51},"height":null,"hideTimeOverride":false,"id":18,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The memory usage of each RisingWave component.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":19,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":20,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (instance) + sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Memory Usage (Total)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":21,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(actor_memory_usage[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"streaming actor - {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_meta_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage meta cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_block_cache_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage block cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(uploading_memory_size{job=~\"$job\",instance=~\"$node\"}) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"storage write buffer - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(stream_memory_usage{job=~\"$job\",instance=~\"$node\"} * on(table_id, actor_id) group_left(materialized_view_id) table_info) by (materialized_view_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized_view {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Memory Usage (Detailed)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Executor cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":22,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Join - cache miss - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Join - total lookups - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n appendonly - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Group top n appendonly - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lookup executor - cache miss - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lookup executor - total lookups - table {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal join - cache miss - table_id {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Temporal join - total lookups - table_id {{table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Materialize - cache hit count - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Materialize - total cache count - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":23,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_join_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id) ) / (sum(rate(stream_join_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (side, join_table_id, degree_table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"join executor cache miss ratio - - {{side}} side, join_table_id {{join_table_id}} degree_table_id {{degree_table_id}} actor {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_lookup_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_lookup_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Agg cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_agg_distinct_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_agg_distinct_total_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Distinct agg cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_group_top_n_appendonly_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_group_top_n_appendonly_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream group top n appendonly cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_lookup_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_lookup_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream lookup cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(stream_temporal_join_cache_miss_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_temporal_join_total_query_cache_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Stream temporal join cache miss ratio - table {{table_id}} actor {{actor_id}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(stream_materialize_cache_hit_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id) ) / (sum(rate(stream_materialize_cache_total_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (table_id, actor_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialize executor cache miss ratio - table {{table_id}} - actor {{actor_id}} {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Cache Miss Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":24,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"memory cache - {{table_id}} @ {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts{type='meta_miss',job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total_meta_miss_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage bloom filter statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":25,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_check_bloom_filter_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter total - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_positive_but_non_exist_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Bloom Filer","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Storage file cache statistics","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":26,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage File Cache","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":52},"height":null,"hideTimeOverride":false,"id":27,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Send/Recv throughput per node for streaming exchange","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":28,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_exchange_frag_send_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Send @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_exchange_frag_recv_size{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Recv @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Streming Remote Exchange (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The remote storage read/write throughput per node","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":29,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Remote I/O (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":30,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_exchange_recv_row_number{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Exchange Recv (Rows/s)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Network","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":53},"height":null,"hideTimeOverride":false,"id":31,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\n Objects are classified into 3 groups:\n - not referenced by versions: these object are being deleted from object store.\n - referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n - referenced by current version: these objects are in the latest version.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":32,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_size{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The storage size of each materialized view","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":33,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_materialized_view_stats{metric='materialized_view_total_size',job=~\"$job\",instance=~\"$node\"}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{metric}}, mv id - {{table_id}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"\n Objects are classified into 3 groups:\n - not referenced by versions: these object are being deleted from object store.\n - referenced by non-current versions: these objects are stale (not in the latest version), but those old versions may still be in use (e.g. long-running pinning). Thus those objects cannot be deleted at the moment.\n - referenced by current version: these objects are in the latest version.\n ","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":34,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"not referenced by versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_old_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by non-current versions","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_object_count{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"referenced by current version","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Object Total Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The number of bytes that have been written by compaction.Flush refers to the process of compacting Memtables to SSTables at Level 0.Compaction refers to the process of compacting SSTables at one level to another level.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":35,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Compaction - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes{job=~\"$job\",instance=~\"$node\"}) by (job) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The remote storage read/write throughput","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":36,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Storage Remote I/O (Bytes/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Size statistics for checkpoint","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":37,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job) (rate(state_store_sync_size_sum{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) / sum by(le, job) (rate(state_store_sync_size_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":54},"height":null,"hideTimeOverride":false,"id":38,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":39,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of bytes read by each source per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":40,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(MB/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"The figure shows the number of rows written into each materialized executor actor per second.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":41,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(stream_executor_row_count{executor_identity=~\".*MaterializeExecutor.*\",job=~\"$job\",instance=~\"$node\"}[$__rate_interval]) * on(actor_id) group_left(materialized_view_id, table_name) (group(table_info{table_type=~\"MATERIALIZED_VIEW\",job=~\"$job\",instance=~\"$node\"}) by (actor_id, materialized_view_id, table_name))) by (materialized_view_id, table_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"materialized view {{table_name}} table_id {{materialized_view_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Materialized View Throughput(rows/s)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"Total number of rows that have been read from the backfill operator used by MV on MV","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":42,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_snapshot_read_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Read Snapshot - table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_backfill_upstream_output_row_count{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Upstream - table_id={{table_id}} actor={{actor_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Backfill Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"We first record the total blocking duration(ns) of output buffer of each actor. It shows how much time it takes an actor to process a message, i.e. a barrier, a watermark or rows of data, on average. Then we divide this duration by 1 second and show it as a percentage.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":43,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(rate(stream_actor_output_buffer_blocking_duration_ns{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (fragment_id, downstream_fragment_id) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fragment {{fragment_id}}->{{downstream_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Blocking Time Ratio (Backpressure)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":55},"height":null,"hideTimeOverride":false,"id":44,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":45,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Running query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":46,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Rejected query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":47,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter{job=~\"$job\",instance=~\"$node\"}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Completed query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":48,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":49,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Local Execution Mode","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":50,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":51,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_source_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_type}} @ {{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":52,"interval":"1s","links":[],"maxDataPoints":1000,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(connector_sink_rows_received{job=~\"$job\",instance=~\"$node\"}[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sink={{connector_type}} @ {{sink_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Connector Sink Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Connector Node","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, instance)","description":"Reporting instance of the metric","hide":0,"includeAll":true,"label":"Node","multi":true,"name":"node","options":[],"query":{"query":"label_values(process_cpu_seconds_total, instance)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"},{"current":{"selected":false,"text":"All","value":"__all"},"definition":"label_values(process_cpu_seconds_total, job)","description":"Reporting job of the metric","hide":0,"includeAll":true,"label":"Job","multi":true,"name":"job","options":[],"query":{"query":"label_values(process_cpu_seconds_total, job)","refId":"StandardVariableQuery"},"refresh":2,"regex":"","skipUrlSync":false,"sort":6,"type":"query"}]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dashboard","uid":"Fcy3uV1nz","version":0} diff --git a/src/storage/src/hummock/store/local_hummock_storage.rs b/src/storage/src/hummock/store/local_hummock_storage.rs index 0c92013d20d0c..db4fb7b9d8930 100644 --- a/src/storage/src/hummock/store/local_hummock_storage.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -351,8 +351,13 @@ impl LocalStateStore for LocalHummockStorage { self.table_id.table_id() ); + let table_id_label = self.table_id.table_id().to_string(); if self.spill_offset < MAX_SPILL_TIMES { self.flush(vec![]).await?; + self.stats + .mem_table_spill_counts + .with_label_values(&[table_id_label.as_str()]) + .inc(); } else { tracing::warn!("No mem table spill occurs, the gap epoch exceeds available range."); } diff --git a/src/storage/src/monitor/hummock_state_store_metrics.rs b/src/storage/src/monitor/hummock_state_store_metrics.rs index c2c5d06a4cd15..b226e7cd248af 100644 --- a/src/storage/src/monitor/hummock_state_store_metrics.rs +++ b/src/storage/src/monitor/hummock_state_store_metrics.rs @@ -77,6 +77,7 @@ pub struct HummockStateStoreMetrics { // memory pub mem_table_memory_size: IntGaugeVec, pub mem_table_item_count: IntGaugeVec, + pub mem_table_spill_counts: RelabeledCounterVec, } pub static GLOBAL_HUMMOCK_STATE_STORE_METRICS: OnceLock = OnceLock::new(); @@ -374,6 +375,20 @@ impl HummockStateStoreMetrics { ) .unwrap(); + let mem_table_spill_counts = register_int_counter_vec_with_registry!( + "state_store_mem_table_spill_counts", + "Total number of mem table spill occurs for one table", + &["table_id"], + registry + ) + .unwrap(); + + let mem_table_spill_counts = RelabeledCounterVec::with_metric_level( + MetricLevel::Info, + mem_table_spill_counts, + metric_level, + ); + Self { bloom_filter_true_negative_counts, bloom_filter_check_counts, @@ -400,6 +415,7 @@ impl HummockStateStoreMetrics { uploader_uploading_task_size, mem_table_memory_size, mem_table_item_count, + mem_table_spill_counts, } } From 255d651d6215ecd0b7aa9caa96215a8f7cee600a Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Mon, 13 Nov 2023 12:03:51 +0800 Subject: [PATCH 43/77] feat(source): embed protobuf well known types when using schema registry (#13370) --- Cargo.lock | 1 + ci/scripts/e2e-source-test.sh | 4 +++ e2e_test/schema_registry/pb.py | 4 ++- e2e_test/schema_registry/pb.slt | 8 ++--- e2e_test/schema_registry/protobuf/user.proto | 5 ++- e2e_test/schema_registry/protobuf/user_pb2.py | 13 ++++---- src/connector/Cargo.toml | 1 + src/connector/build.rs | 6 ++++ .../src/parser/protobuf/schema_resolver.rs | 31 +++++++++++++++++++ 9 files changed, 61 insertions(+), 12 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 003ecdb11c842..d662200755408 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7546,6 +7546,7 @@ dependencies = [ "prost-reflect", "prost-types 0.12.1", "protobuf-native", + "protobuf-src", "pulsar", "quote", "rand", diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 05dc731394200..2f9c29d76b957 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -98,6 +98,10 @@ RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=i cargo make ci-start ci-1cn-1fe python3 -m pip install requests protobuf confluent-kafka python3 e2e_test/schema_registry/pb.py "message_queue:29092" "http://message_queue:8081" "sr_pb_test" 20 +echo "make sure google/protobuf/source_context.proto is NOT in schema registry" +curl --silent 'http://message_queue:8081/subjects'; echo +# curl --silent --head -X GET 'http://message_queue:8081/subjects/google%2Fprotobuf%2Fsource_context.proto/versions' | grep 404 +curl --silent 'http://message_queue:8081/subjects' | grep -v 'google/protobuf/source_context.proto' sqllogictest -p 4566 -d dev './e2e_test/schema_registry/pb.slt' echo "--- Kill cluster" diff --git a/e2e_test/schema_registry/pb.py b/e2e_test/schema_registry/pb.py index 8b5e850a82796..e83fd1d36f8a5 100644 --- a/e2e_test/schema_registry/pb.py +++ b/e2e_test/schema_registry/pb.py @@ -1,4 +1,5 @@ from protobuf import user_pb2 +from google.protobuf.source_context_pb2 import SourceContext import sys from confluent_kafka import Producer from confluent_kafka.serialization import ( @@ -21,6 +22,7 @@ def get_user(i): address="Address_{}".format(i), city="City_{}".format(i), gender=user_pb2.MALE if i % 2 == 0 else user_pb2.FEMALE, + sc=SourceContext(file_name="source/context_{:03}.proto".format(i)), ) @@ -29,7 +31,7 @@ def send_to_kafka(producer_conf, schema_registry_conf, topic, num_records): serializer = ProtobufSerializer( user_pb2.User, schema_registry_client, - {"use.deprecated.format": False}, + {"use.deprecated.format": False, 'skip.known.types': True}, ) producer = Producer(producer_conf) diff --git a/e2e_test/schema_registry/pb.slt b/e2e_test/schema_registry/pb.slt index 9914d3fa04b2f..618ecd71e960c 100644 --- a/e2e_test/schema_registry/pb.slt +++ b/e2e_test/schema_registry/pb.slt @@ -39,14 +39,14 @@ select count(*) from sr_pb_test; ---- 20 -query II -select min(id), max(id) from sr_pb_test; +query IIT +select min(id), max(id), max((sc).file_name) from sr_pb_test; ---- -0 19 +0 19 source/context_019.proto statement ok drop table sr_pb_test; statement ok -drop table sr_pb_test_bk; \ No newline at end of file +drop table sr_pb_test_bk; diff --git a/e2e_test/schema_registry/protobuf/user.proto b/e2e_test/schema_registry/protobuf/user.proto index 79245a3ba4ff9..e6c5f109bbd76 100644 --- a/e2e_test/schema_registry/protobuf/user.proto +++ b/e2e_test/schema_registry/protobuf/user.proto @@ -2,15 +2,18 @@ syntax = "proto3"; package test; +import "google/protobuf/source_context.proto"; + message User { int32 id = 1; string name = 2; string address = 3; string city = 4; Gender gender = 5; + google.protobuf.SourceContext sc = 6; } enum Gender { MALE = 0; FEMALE = 1; -} \ No newline at end of file +} diff --git a/e2e_test/schema_registry/protobuf/user_pb2.py b/e2e_test/schema_registry/protobuf/user_pb2.py index 1258739d4ae2f..b87f3a5ea1d81 100644 --- a/e2e_test/schema_registry/protobuf/user_pb2.py +++ b/e2e_test/schema_registry/protobuf/user_pb2.py @@ -1,6 +1,7 @@ # -*- coding: utf-8 -*- # Generated by the protocol buffer compiler. DO NOT EDIT! # source: user.proto +# Protobuf Python Version: 4.25.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -11,18 +12,18 @@ _sym_db = _symbol_database.Default() +from google.protobuf import source_context_pb2 as google_dot_protobuf_dot_source__context__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\nuser.proto\x12\x04test\"]\n\x04User\x12\n\n\x02id\x18\x01 \x01(\x05\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x0f\n\x07\x61\x64\x64ress\x18\x03 \x01(\t\x12\x0c\n\x04\x63ity\x18\x04 \x01(\t\x12\x1c\n\x06gender\x18\x05 \x01(\x0e\x32\x0c.test.Gender*\x1e\n\x06Gender\x12\x08\n\x04MALE\x10\x00\x12\n\n\x06\x46\x45MALE\x10\x01\x62\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\nuser.proto\x12\x04test\x1a$google/protobuf/source_context.proto\"\x89\x01\n\x04User\x12\n\n\x02id\x18\x01 \x01(\x05\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x0f\n\x07\x61\x64\x64ress\x18\x03 \x01(\t\x12\x0c\n\x04\x63ity\x18\x04 \x01(\t\x12\x1c\n\x06gender\x18\x05 \x01(\x0e\x32\x0c.test.Gender\x12*\n\x02sc\x18\x06 \x01(\x0b\x32\x1e.google.protobuf.SourceContext*\x1e\n\x06Gender\x12\x08\n\x04MALE\x10\x00\x12\n\n\x06\x46\x45MALE\x10\x01\x62\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) _builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'user_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_GENDER']._serialized_start=115 - _globals['_GENDER']._serialized_end=145 - _globals['_USER']._serialized_start=20 - _globals['_USER']._serialized_end=113 + _globals['_GENDER']._serialized_start=198 + _globals['_GENDER']._serialized_end=228 + _globals['_USER']._serialized_start=59 + _globals['_USER']._serialized_end=196 # @@protoc_insertion_point(module_scope) diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index fe31e1a5c6bc4..3e538d7f2eebd 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -154,6 +154,7 @@ walkdir = "2" [build-dependencies] prost-build = "0.12" +protobuf-src = "1" [[bench]] name = "parser" diff --git a/src/connector/build.rs b/src/connector/build.rs index 3ace772d46039..1143f67d5538a 100644 --- a/src/connector/build.rs +++ b/src/connector/build.rs @@ -26,4 +26,10 @@ fn main() { .out_dir("./src/parser/protobuf") .compile_protos(&protos, &Vec::::new()) .unwrap(); + + let proto_include_path = protobuf_src::include(); + println!( + "cargo:rustc-env=PROTO_INCLUDE={}", + proto_include_path.to_str().unwrap() + ); } diff --git a/src/connector/src/parser/protobuf/schema_resolver.rs b/src/connector/src/parser/protobuf/schema_resolver.rs index 841c6fee825ad..bc41062c4b9c7 100644 --- a/src/connector/src/parser/protobuf/schema_resolver.rs +++ b/src/connector/src/parser/protobuf/schema_resolver.rs @@ -34,6 +34,31 @@ pub(super) async fn load_file_descriptor_from_http(location: &Url) -> Result { + &[$( + ( + concat!("google/protobuf/", $path), + include_bytes!(concat!(env!("PROTO_INCLUDE"), "/google/protobuf/", $path)).as_slice(), + ) + ),+] + }; +} +const WELL_KNOWN_TYPES: &[(&str, &[u8])] = embed_wkts![ + "any.proto", + "api.proto", + "compiler/plugin.proto", + "descriptor.proto", + "duration.proto", + "empty.proto", + "field_mask.proto", + "source_context.proto", + "struct.proto", + "timestamp.proto", + "type.proto", + "wrappers.proto", +]; + // Pull protobuf schema and all it's deps from the confluent schema registry, // and compile then into one file descriptor pub(super) async fn compile_file_descriptor_from_schema_registry( @@ -51,6 +76,12 @@ pub(super) async fn compile_file_descriptor_from_schema_registry( subject.schema.content.as_bytes().to_vec(), ); } + for (path, bytes) in WELL_KNOWN_TYPES { + source_tree + .as_mut() + .add_file(Path::new(path), bytes.to_vec()); + } + let mut error_collector = SimpleErrorCollector::new(); // `db` needs to be dropped before we can iterate on `error_collector`. let fds = { From b10238c4a4c07f3651d4a5e06aa23705801f8fb9 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Mon, 13 Nov 2023 12:28:38 +0800 Subject: [PATCH 44/77] fix(s3): more accurate expiration judgment when starting the cluster (#13371) --- src/meta/src/hummock/manager/mod.rs | 2 +- src/object_store/src/object/s3.rs | 23 ++++++++++++++++++++--- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 47c27d758d79e..a578cb571c706 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -352,7 +352,7 @@ impl HummockManager { if let risingwave_object_store::object::ObjectStoreImpl::S3(s3) = object_store.as_ref() && !env.opts.do_not_config_object_storage_lifecycle { - let is_bucket_expiration_configured = s3.inner().configure_bucket_lifecycle().await; + let is_bucket_expiration_configured = s3.inner().configure_bucket_lifecycle(state_store_dir).await; if is_bucket_expiration_configured { return Err(ObjectError::internal("Cluster cannot start with object expiration configured for bucket because RisingWave data will be lost when object expiration kicks in. Please disable object expiration and restart the cluster.") diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 6dce56ed4c759..71592f45b4d2b 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -705,7 +705,7 @@ impl S3ObjectStore { /// - /// - MinIO /// - - pub async fn configure_bucket_lifecycle(&self) -> bool { + pub async fn configure_bucket_lifecycle(&self, data_directory: &str) -> bool { // Check if lifecycle is already configured to avoid overriding existing configuration. let bucket = self.bucket.as_str(); let mut configured_rules = vec![]; @@ -716,11 +716,28 @@ impl S3ObjectStore { .send() .await; let mut is_expiration_configured = false; + if let Ok(config) = &get_config_result { for rule in config.rules().unwrap_or_default() { - if rule.expiration().is_some() { - is_expiration_configured = true; + // Check if the filter is not set or the prifix in the filter is data directory in RisingWave, + // and if the expiration status of the rule is "Enabled". + // If both conditions are met, it is considered that there is a risk of data deletion. + match rule.filter().as_ref() { + Some(&LifecycleRuleFilter::Prefix(prefix)) => { + if let Some(ExpirationStatus::Enabled) = rule.status && data_directory.starts_with(prefix){ + is_expiration_configured = true; + } + } + None => { + if let Some(ExpirationStatus::Enabled) = rule.status { + is_expiration_configured = true; + } + } + _ => { + + } } + if matches!(rule.status().unwrap(), ExpirationStatus::Enabled) && rule.abort_incomplete_multipart_upload().is_some() { From 0ec7c6e8847f1762c71860d3d87c3d7862a86595 Mon Sep 17 00:00:00 2001 From: Wallace Date: Mon, 13 Nov 2023 15:55:24 +0800 Subject: [PATCH 45/77] fix(storage): avoid preload all range-tombstone before compaction job start (#12792) Signed-off-by: Little-Wallace --- src/storage/benches/bench_compactor.rs | 8 +- src/storage/hummock_sdk/src/compact.rs | 14 +- .../hummock_test/src/compactor_tests.rs | 9 +- .../src/hummock/compactor/compactor_runner.rs | 150 +++--- src/storage/src/hummock/compactor/mod.rs | 23 +- .../compactor/shared_buffer_compact.rs | 94 ++-- .../iterator/concat_delete_range_iterator.rs | 56 ++- .../hummock/iterator/delete_range_iterator.rs | 19 + .../sstable/delete_range_aggregator.rs | 468 +++++++----------- src/storage/src/hummock/sstable/mod.rs | 51 +- .../src/hummock/sstable/multi_builder.rs | 106 ++-- src/storage/src/hummock/test_utils.rs | 40 +- 12 files changed, 535 insertions(+), 503 deletions(-) diff --git a/src/storage/benches/bench_compactor.rs b/src/storage/benches/bench_compactor.rs index 41a3649adc5cf..9e48c2093328a 100644 --- a/src/storage/benches/bench_compactor.rs +++ b/src/storage/benches/bench_compactor.rs @@ -22,6 +22,7 @@ use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::HummockEpoch; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::{InMemObjectStore, ObjectStore, ObjectStoreImpl}; use risingwave_pb::hummock::{compact_task, SstableInfo}; @@ -30,7 +31,8 @@ use risingwave_storage::hummock::compactor::{ ConcatSstableIterator, DummyCompactionFilter, TaskConfig, TaskProgress, }; use risingwave_storage::hummock::iterator::{ - ConcatIterator, Forward, HummockIterator, UnorderedMergeIteratorInner, + ConcatIterator, Forward, ForwardMergeRangeIterator, HummockIterator, + UnorderedMergeIteratorInner, }; use risingwave_storage::hummock::multi_builder::{ CapacitySplitTableBuilder, LocalTableBuilderFactory, @@ -39,7 +41,7 @@ use risingwave_storage::hummock::sstable::SstableIteratorReadOptions; use risingwave_storage::hummock::sstable_store::SstableStoreRef; use risingwave_storage::hummock::value::HummockValue; use risingwave_storage::hummock::{ - CachePolicy, CompactionDeleteRanges, FileCache, SstableBuilder, SstableBuilderOptions, + CachePolicy, CompactionDeleteRangeIterator, FileCache, SstableBuilder, SstableBuilderOptions, SstableIterator, SstableStore, SstableWriterOptions, Xor16FilterBuilder, }; use risingwave_storage::monitor::{CompactorMetrics, StoreLocalStatistic}; @@ -196,7 +198,7 @@ async fn compact>(iter: I, sstable_store }; compact_and_build_sst( &mut builder, - Arc::new(CompactionDeleteRanges::default()), + CompactionDeleteRangeIterator::new(ForwardMergeRangeIterator::new(HummockEpoch::MAX)), &task_config, Arc::new(CompactorMetrics::unused()), iter, diff --git a/src/storage/hummock_sdk/src/compact.rs b/src/storage/hummock_sdk/src/compact.rs index 4d590a558d79d..c3d1777df2dcf 100644 --- a/src/storage/hummock_sdk/src/compact.rs +++ b/src/storage/hummock_sdk/src/compact.rs @@ -153,21 +153,25 @@ pub fn estimate_memory_for_compact_task( // The common size of SstableMeta in tests is no more than 1m (mainly from xor filters). Even // though SstableMeta is used for a shorter period of time, it is safe to use 3m for the // calculation. - // TODO: Note that this algorithm may fail when SstableMeta is occupied by a large number of - // range tombstones const ESTIMATED_META_SIZE: u64 = 3 * 1048576; // The memory usage of the SstableStreamIterator comes from SstableInfo with some state // information (use ESTIMATED_META_SIZE to estimate it), the BlockStream being read (one block), // and tcp recv_buffer_size. - let max_input_stream_estimated_memory = ESTIMATED_META_SIZE + block_size + recv_buffer_size; + let max_input_stream_estimated_memory = block_size + recv_buffer_size; // input for level in &task.input_ssts { if level.level_type() == LevelType::Nonoverlapping { - result += max_input_stream_estimated_memory; + let mut meta_size = 0; + for sst in &level.table_infos { + meta_size = std::cmp::max(meta_size, sst.file_size - sst.meta_offset); + } + result += max_input_stream_estimated_memory + meta_size; } else { - result += max_input_stream_estimated_memory * level.table_infos.len() as u64; + for sst in &level.table_infos { + result += max_input_stream_estimated_memory + sst.file_size - sst.meta_offset; + } } } diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index cd921b3725369..6edb0ba79d11e 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -62,10 +62,9 @@ pub(crate) mod tests { use risingwave_storage::hummock::test_utils::gen_test_sstable_info; use risingwave_storage::hummock::value::HummockValue; use risingwave_storage::hummock::{ - CachePolicy, CompactionDeleteRanges, CompressionAlgorithm, - HummockStorage as GlobalHummockStorage, HummockStorage, MemoryLimiter, - SharedComapctorObjectIdManager, Sstable, SstableBuilderOptions, SstableIteratorReadOptions, - SstableObjectIdManager, + CachePolicy, CompressionAlgorithm, HummockStorage as GlobalHummockStorage, HummockStorage, + MemoryLimiter, SharedComapctorObjectIdManager, Sstable, SstableBuilderOptions, + SstableIteratorReadOptions, SstableObjectIdManager, }; use risingwave_storage::monitor::{CompactorMetrics, StoreLocalStatistic}; use risingwave_storage::opts::StorageOpts; @@ -1421,7 +1420,6 @@ pub(crate) mod tests { gc_delete_keys: true, ..Default::default() }; - let deg = Arc::new(CompactionDeleteRanges::default()); let multi_filter_key_extractor = Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)); let compaction_filter = DummyCompactionFilter {}; @@ -1446,7 +1444,6 @@ pub(crate) mod tests { .run( compaction_filter, multi_filter_key_extractor, - deg, Arc::new(TaskProgress::default()), ) .await diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 1fcd519179e40..591d79cd7d6b8 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -28,7 +28,7 @@ use risingwave_hummock_sdk::key_range::{KeyRange, KeyRangeCommon}; use risingwave_hummock_sdk::table_stats::{add_table_stats_map, TableStats, TableStatsMap}; use risingwave_hummock_sdk::{can_concat, EpochWithGap}; use risingwave_pb::hummock::compact_task::{TaskStatus, TaskType}; -use risingwave_pb::hummock::{BloomFilterType, CompactTask, LevelType, SstableInfo}; +use risingwave_pb::hummock::{BloomFilterType, CompactTask, LevelType}; use tokio::sync::oneshot::Receiver; use super::task_progress::TaskProgress; @@ -42,13 +42,15 @@ use crate::hummock::compactor::task_progress::TaskProgressGuard; use crate::hummock::compactor::{ fast_compactor_runner, CompactOutput, CompactionFilter, Compactor, CompactorContext, }; -use crate::hummock::iterator::{Forward, HummockIterator, UnorderedMergeIteratorInner}; +use crate::hummock::iterator::{ + Forward, ForwardMergeRangeIterator, HummockIterator, UnorderedMergeIteratorInner, +}; use crate::hummock::multi_builder::{CapacitySplitTableBuilder, TableBuilderFactory}; -use crate::hummock::sstable::CompactionDeleteRangesBuilder; use crate::hummock::value::HummockValue; use crate::hummock::{ - BlockedXor16FilterBuilder, CachePolicy, CompactionDeleteRanges, CompressionAlgorithm, - GetObjectId, HummockResult, MonotonicDeleteEvent, SstableBuilderOptions, SstableStoreRef, + BlockedXor16FilterBuilder, CachePolicy, CompactionDeleteRangeIterator, CompressionAlgorithm, + GetObjectId, HummockResult, MonotonicDeleteEvent, SstableBuilderOptions, + SstableDeleteRangeIterator, SstableStoreRef, }; use crate::monitor::{CompactorMetrics, StoreLocalStatistic}; @@ -122,16 +124,15 @@ impl CompactorRunner { &self, compaction_filter: impl CompactionFilter, filter_key_extractor: Arc, - del_agg: Arc, task_progress: Arc, ) -> HummockResult { - let iter = self.build_sst_iter(task_progress.clone())?; + let (iter, del_iter) = self.build_sst_iter(task_progress.clone()).await?; let (ssts, compaction_stat) = self .compactor .compact_key_range( iter, compaction_filter, - del_agg, + del_iter, filter_key_extractor, Some(task_progress), Some(self.compact_task.task_id), @@ -141,46 +142,22 @@ impl CompactorRunner { Ok((self.split_index, ssts, compaction_stat)) } - pub async fn build_delete_range_iter( - sstable_infos: &Vec, - sstable_store: &SstableStoreRef, - filter: &mut F, - ) -> HummockResult> { - let mut builder = CompactionDeleteRangesBuilder::default(); - let mut local_stats = StoreLocalStatistic::default(); - - for table_info in sstable_infos { - if table_info.range_tombstone_count > 0 { - let table = sstable_store.sstable(table_info, &mut local_stats).await?; - let mut range_tombstone_list = - table.value().meta.monotonic_tombstone_events.clone(); - range_tombstone_list.iter_mut().for_each(|tombstone| { - if filter.should_delete(FullKey::from_user_key( - tombstone.event_key.left_user_key.as_ref(), - tombstone.new_epoch, - )) { - tombstone.new_epoch = MAX_EPOCH; - } - }); - builder.add_delete_events(range_tombstone_list); - } - } - - let aggregator = builder.build_for_compaction(); - Ok(aggregator) - } - /// Build the merge iterator based on the given input ssts. - fn build_sst_iter( + async fn build_sst_iter( &self, task_progress: Arc, - ) -> HummockResult> { + ) -> HummockResult<( + impl HummockIterator, + CompactionDeleteRangeIterator, + )> { let mut table_iters = Vec::new(); + let mut local_stats = StoreLocalStatistic::default(); let compact_io_retry_time = self .compactor .context .storage_opts .compact_iter_recreate_timeout_ms; + let mut del_iter = ForwardMergeRangeIterator::new(MAX_EPOCH); for level in &self.compact_task.input_ssts { if level.table_infos.is_empty() { @@ -204,6 +181,14 @@ impl CompactorRunner { }) .cloned() .collect_vec(); + let delete_range_ssts = tables + .iter() + .filter(|sst| sst.range_tombstone_count > 0) + .cloned() + .collect_vec(); + if !delete_range_ssts.is_empty() { + del_iter.add_concat_iter(delete_range_ssts, self.sstable_store.clone()); + } table_iters.push(ConcatSstableIterator::new( self.compact_task.existing_table_ids.clone(), tables, @@ -223,6 +208,13 @@ impl CompactorRunner { if !self.key_range.full_key_overlap(&key_range) || !exist_table { continue; } + if table_info.range_tombstone_count > 0 { + let table = self + .sstable_store + .sstable(table_info, &mut local_stats) + .await?; + del_iter.add_sst_iter(SstableDeleteRangeIterator::new(table)); + } table_iters.push(ConcatSstableIterator::new( self.compact_task.existing_table_ids.clone(), vec![table_info.clone()], @@ -234,7 +226,10 @@ impl CompactorRunner { } } } - Ok(UnorderedMergeIteratorInner::for_compactor(table_iters)) + Ok(( + UnorderedMergeIteratorInner::for_compactor(table_iters), + CompactionDeleteRangeIterator::new(del_iter), + )) } } @@ -299,7 +294,7 @@ pub async fn compact( ]) .start_timer(); - let mut multi_filter = build_multi_compaction_filter(&compact_task); + let multi_filter = build_multi_compaction_filter(&compact_task); let mut compact_table_ids = compact_task .input_ssts @@ -405,20 +400,6 @@ pub async fn compact( let mut abort_handles = vec![]; let task_progress_guard = TaskProgressGuard::new(compact_task.task_id, context.task_progress_manager.clone()); - let delete_range_agg = match CompactorRunner::build_delete_range_iter( - &sstable_infos, - &compactor_context.sstable_store, - &mut multi_filter, - ) - .await - { - Ok(agg) => agg, - Err(err) => { - tracing::warn!("Failed to build delete range aggregator {:#?}", err); - task_status = TaskStatus::ExecuteFailed; - return compact_done(compact_task, context.clone(), vec![], task_status); - } - }; let capacity = estimate_task_output_capacity(context.clone(), &compact_task); @@ -517,11 +498,10 @@ pub async fn compact( compact_task.clone(), object_id_getter.clone(), ); - let del_agg = delete_range_agg.clone(); let task_progress = task_progress_guard.progress.clone(); let runner = async move { compactor_runner - .run(filter, multi_filter_key_extractor, del_agg, task_progress) + .run(filter, multi_filter_key_extractor, task_progress) .await }; let traced = match context.await_tree_reg.as_ref() { @@ -657,7 +637,7 @@ fn compact_done( pub async fn compact_and_build_sst( sst_builder: &mut CapacitySplitTableBuilder, - del_agg: Arc, + mut del_iter: CompactionDeleteRangeIterator, task_config: &TaskConfig, compactor_metrics: Arc, mut iter: impl HummockIterator, @@ -667,13 +647,12 @@ pub async fn compact_and_build_sst( where F: TableBuilderFactory, { - let mut del_iter = del_agg.iter(); if !task_config.key_range.left.is_empty() { let full_key = FullKey::decode(&task_config.key_range.left); iter.seek(full_key) .verbose_instrument_await("iter_seek") .await?; - del_iter.seek(full_key.user_key); + del_iter.seek(full_key.user_key).await?; if !task_config.gc_delete_keys && del_iter.is_valid() && del_iter.earliest_epoch() != MAX_EPOCH @@ -687,7 +666,7 @@ where } } else { iter.rewind().verbose_instrument_await("rewind").await?; - del_iter.rewind(); + del_iter.rewind().await?; }; let end_key = if task_config.key_range.right.is_empty() { @@ -754,16 +733,17 @@ where let target_extended_user_key = PointRange::from_user_key(iter_key.user_key, false); while del_iter.is_valid() && del_iter.key().as_ref().le(&target_extended_user_key) { - del_iter.update_range(); + let event_key = del_iter.key().to_vec(); + del_iter.next().await?; if !task_config.gc_delete_keys { sst_builder .add_monotonic_delete(MonotonicDeleteEvent { - event_key: del_iter.key().clone(), new_epoch: del_iter.earliest_epoch(), + event_key, }) .await?; } - del_iter.next(); + progress_key_num += 1; if let Some(task_progress) = task_progress.as_ref() && progress_key_num >= PROGRESS_KEY_INTERVAL @@ -854,10 +834,10 @@ where if !task_config.gc_delete_keys { let extended_largest_user_key = PointRange::from_user_key(end_key.user_key.clone(), false); + + let end_key_ref = extended_largest_user_key.as_ref(); while del_iter.is_valid() { - if !extended_largest_user_key.is_empty() - && del_iter.key().ge(&extended_largest_user_key) - { + if !end_key_ref.is_empty() && del_iter.key().ge(&end_key_ref) { sst_builder .add_monotonic_delete(MonotonicDeleteEvent { event_key: extended_largest_user_key, @@ -866,14 +846,14 @@ where .await?; break; } - del_iter.update_range(); + let event_key = del_iter.key().to_vec(); + del_iter.next().await?; sst_builder .add_monotonic_delete(MonotonicDeleteEvent { - event_key: del_iter.key().clone(), new_epoch: del_iter.earliest_epoch(), + event_key, }) .await?; - del_iter.next(); progress_key_num += 1; if let Some(task_progress) = task_progress.as_ref() && progress_key_num >= PROGRESS_KEY_INTERVAL @@ -981,17 +961,25 @@ mod tests { .cloned() .collect_vec(); - let collector = CompactorRunner::build_delete_range_iter( - &sstable_infos, - &sstable_store, - &mut state_clean_up_filter, - ) - .await - .unwrap(); - let ret = collector.get_tombstone_between( - UserKey::::default().as_ref(), - UserKey::::default().as_ref(), - ); + let mut iter = ForwardMergeRangeIterator::new(MAX_EPOCH); + iter.add_concat_iter(sstable_infos, sstable_store); + + let ret = CompactionDeleteRangeIterator::new(iter) + .get_tombstone_between( + UserKey::::default().as_ref(), + UserKey::::default().as_ref(), + ) + .await + .unwrap(); + let ret = ret + .into_iter() + .filter(|event| { + !state_clean_up_filter.should_delete(FullKey::from_user_key( + event.event_key.left_user_key.as_ref(), + event.new_epoch, + )) + }) + .collect_vec(); assert_eq!( ret, diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index 137682d6f7825..274020b2013c9 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -69,8 +69,7 @@ pub use self::compaction_utils::{CompactionStatistics, RemoteBuilderFactory, Tas pub use self::task_progress::TaskProgress; use super::multi_builder::CapacitySplitTableBuilder; use super::{ - CompactionDeleteRanges, GetObjectId, HummockResult, SstableBuilderOptions, - SstableObjectIdManager, Xor16FilterBuilder, + GetObjectId, HummockResult, SstableBuilderOptions, SstableObjectIdManager, Xor16FilterBuilder, }; use crate::filter_key_extractor::{ FilterKeyExtractorImpl, FilterKeyExtractorManager, StaticFilterKeyExtractorManager, @@ -80,9 +79,9 @@ use crate::hummock::iterator::{Forward, HummockIterator}; use crate::hummock::multi_builder::SplitTableOutput; use crate::hummock::vacuum::Vacuum; use crate::hummock::{ - validate_ssts, BatchSstableWriterFactory, BlockedXor16FilterBuilder, FilterBuilder, - HummockError, SharedComapctorObjectIdManager, SstableWriterFactory, - StreamingSstableWriterFactory, + validate_ssts, BatchSstableWriterFactory, BlockedXor16FilterBuilder, + CompactionDeleteRangeIterator, FilterBuilder, HummockError, SharedComapctorObjectIdManager, + SstableWriterFactory, StreamingSstableWriterFactory, }; use crate::monitor::CompactorMetrics; @@ -123,7 +122,7 @@ impl Compactor { &self, iter: impl HummockIterator, compaction_filter: impl CompactionFilter, - del_agg: Arc, + del_iter: CompactionDeleteRangeIterator, filter_key_extractor: Arc, task_progress: Option>, task_id: Option, @@ -154,7 +153,7 @@ impl Compactor { factory, iter, compaction_filter, - del_agg, + del_iter, filter_key_extractor, task_progress.clone(), self.object_id_getter.clone(), @@ -166,7 +165,7 @@ impl Compactor { factory, iter, compaction_filter, - del_agg, + del_iter, filter_key_extractor, task_progress.clone(), self.object_id_getter.clone(), @@ -181,7 +180,7 @@ impl Compactor { factory, iter, compaction_filter, - del_agg, + del_iter, filter_key_extractor, task_progress.clone(), self.object_id_getter.clone(), @@ -193,7 +192,7 @@ impl Compactor { factory, iter, compaction_filter, - del_agg, + del_iter, filter_key_extractor, task_progress.clone(), self.object_id_getter.clone(), @@ -278,7 +277,7 @@ impl Compactor { writer_factory: F, iter: impl HummockIterator, compaction_filter: impl CompactionFilter, - del_agg: Arc, + del_iter: CompactionDeleteRangeIterator, filter_key_extractor: Arc, task_progress: Option>, object_id_getter: Box, @@ -304,7 +303,7 @@ impl Compactor { ); let compaction_statistics = compact_and_build_sst( &mut sst_builder, - del_agg, + del_iter, &self.task_config, self.context.compactor_metrics.clone(), iter, diff --git a/src/storage/src/hummock/compactor/shared_buffer_compact.rs b/src/storage/src/hummock/compactor/shared_buffer_compact.rs index 0aae5dbbd9310..7d75c9adf6d49 100644 --- a/src/storage/src/hummock/compactor/shared_buffer_compact.rs +++ b/src/storage/src/hummock/compactor/shared_buffer_compact.rs @@ -25,7 +25,7 @@ use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::MAX_EPOCH; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; -use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKey}; +use risingwave_hummock_sdk::key::{FullKey, PointRange, TableKey, UserKey}; use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::{CompactionGroupId, EpochWithGap, LocalSstableInfo}; use risingwave_pb::hummock::compact_task; @@ -37,16 +37,17 @@ use crate::hummock::compactor::context::CompactorContext; use crate::hummock::compactor::{CompactOutput, Compactor}; use crate::hummock::event_handler::uploader::UploadTaskPayload; use crate::hummock::event_handler::LocalInstanceId; -use crate::hummock::iterator::{Forward, HummockIterator, OrderedMergeIteratorInner}; +use crate::hummock::iterator::{ + Forward, ForwardMergeRangeIterator, HummockIterator, OrderedMergeIteratorInner, +}; use crate::hummock::shared_buffer::shared_buffer_batch::{ SharedBufferBatch, SharedBufferBatchInner, SharedBufferVersionedEntry, }; -use crate::hummock::sstable::CompactionDeleteRangesBuilder; use crate::hummock::utils::MemoryTracker; use crate::hummock::value::HummockValue; use crate::hummock::{ - create_monotonic_events_from_compaction_delete_events, BlockedXor16FilterBuilder, CachePolicy, - CompactionDeleteRanges, GetObjectId, HummockError, HummockResult, SstableBuilderOptions, + BlockedXor16FilterBuilder, CachePolicy, CompactionDeleteRangeIterator, GetObjectId, + HummockError, HummockResult, MonotonicDeleteEvent, SstableBuilderOptions, SstableObjectIdManagerRef, }; use crate::mem_table::ImmutableMemtable; @@ -138,7 +139,6 @@ async fn compact_shared_buffer( let mut size_and_start_user_keys = vec![]; let mut compact_data_size = 0; - let mut builder = CompactionDeleteRangesBuilder::default(); payload.retain(|imm| { let ret = existing_table_ids.contains(&imm.table_id.table_id); if !ret { @@ -151,8 +151,6 @@ async fn compact_shared_buffer( }); let mut total_key_count = 0; for imm in &payload { - let tombstones = imm.get_delete_range_tombstones(); - builder.add_delete_events(tombstones); total_key_count += imm.kv_count(); let data_size = { // calculate encoded bytes of key var length @@ -232,7 +230,6 @@ async fn compact_shared_buffer( let mut compaction_futures = vec![]; let use_block_based_filter = BlockedXor16FilterBuilder::is_kv_count_too_large(total_key_count); - let agg = builder.build_for_compaction(); for (split_index, key_range) in splits.into_iter().enumerate() { let compactor = SharedBufferCompactRunner::new( split_index, @@ -243,15 +240,21 @@ async fn compact_shared_buffer( use_block_based_filter, Box::new(sstable_object_id_manager.clone()), ); - let iter = OrderedMergeIteratorInner::new( - payload.iter().map(|imm| imm.clone().into_forward_iter()), - ); + let mut forward_iters = Vec::with_capacity(payload.len()); + let mut del_iter = ForwardMergeRangeIterator::new(MAX_EPOCH); + for imm in &payload { + forward_iters.push(imm.clone().into_forward_iter()); + del_iter.add_batch_iter(imm.delete_range_iter()); + } let compaction_executor = context.compaction_executor.clone(); let multi_filter_key_extractor = multi_filter_key_extractor.clone(); - let del_range_agg = agg.clone(); let handle = compaction_executor.spawn(async move { compactor - .run(iter, multi_filter_key_extractor, del_range_agg) + .run( + OrderedMergeIteratorInner::new(forward_iters), + multi_filter_key_extractor, + CompactionDeleteRangeIterator::new(del_iter), + ) .await }); compaction_futures.push(handle); @@ -319,7 +322,7 @@ pub async fn merge_imms_in_memory( let mut largest_table_key = Bound::Included(Bytes::new()); let mut imm_iters = Vec::with_capacity(imms.len()); - let mut builder = CompactionDeleteRangesBuilder::default(); + let mut del_iter = ForwardMergeRangeIterator::new(MAX_EPOCH); for imm in imms { assert!( imm.kv_count() > 0 || imm.has_range_tombstone(), @@ -335,7 +338,7 @@ pub async fn merge_imms_in_memory( epochs.push(imm.min_epoch()); kv_count += imm.kv_count(); merged_size += imm.size(); - builder.add_delete_events(imm.get_delete_range_tombstones()); + del_iter.add_batch_iter(imm.delete_range_iter()); if smallest_empty || smallest_table_key.as_ref().gt(imm.raw_smallest_key()) { smallest_table_key.clear(); @@ -358,9 +361,8 @@ pub async fn merge_imms_in_memory( imm_iters.push(imm.into_forward_iter()); } - let compaction_delete_ranges = builder.build_for_compaction(); - let mut del_iter = compaction_delete_ranges.iter(); - del_iter.rewind(); + let mut del_iter = CompactionDeleteRangeIterator::new(del_iter); + del_iter.rewind().await?; epochs.sort(); // use merge iterator to merge input imms @@ -378,28 +380,42 @@ pub async fn merge_imms_in_memory( .first() .map(|((k, _), _)| k.clone()) .unwrap_or_default(); - del_iter.earliest_delete_which_can_see_key( - UserKey::new(table_id, TableKey(pivot.as_ref())), - MAX_EPOCH, - ); + let mut monotonic_tombstone_events = vec![]; + let target_extended_user_key = + PointRange::from_user_key(UserKey::new(table_id, TableKey(pivot.as_ref())), false); + while del_iter.is_valid() && del_iter.key().le(&target_extended_user_key) { + let event_key = del_iter.key().to_vec(); + del_iter.next().await?; + monotonic_tombstone_events.push(MonotonicDeleteEvent { + event_key, + new_epoch: del_iter.earliest_epoch(), + }); + } + let mut versions: Vec<(EpochWithGap, HummockValue)> = Vec::new(); let mut pivot_last_delete_epoch = MAX_EPOCH; for ((key, value), epoch) in items { assert!(key >= pivot, "key should be in ascending order"); - let earliest_range_delete_which_can_see_key = if key == pivot { - del_iter.earliest_delete_since(epoch.pure_epoch()) - } else { + if key != pivot { merged_payload.push((pivot, versions)); pivot = key; pivot_last_delete_epoch = MAX_EPOCH; versions = vec![]; - del_iter.earliest_delete_which_can_see_key( - UserKey::new(table_id, TableKey(pivot.as_ref())), - epoch.pure_epoch(), - ) - }; + let target_extended_user_key = + PointRange::from_user_key(UserKey::new(table_id, TableKey(pivot.as_ref())), false); + while del_iter.is_valid() && del_iter.key().le(&target_extended_user_key) { + let event_key = del_iter.key().to_vec(); + del_iter.next().await?; + monotonic_tombstone_events.push(MonotonicDeleteEvent { + event_key, + new_epoch: del_iter.earliest_epoch(), + }); + } + } + let earliest_range_delete_which_can_see_key = + del_iter.earliest_delete_since(epoch.pure_epoch()); if value.is_delete() { pivot_last_delete_epoch = epoch.pure_epoch(); } else if earliest_range_delete_which_can_see_key < pivot_last_delete_epoch { @@ -420,15 +436,21 @@ pub async fn merge_imms_in_memory( } versions.push((epoch, value)); } + while del_iter.is_valid() { + let event_key = del_iter.key().to_vec(); + del_iter.next().await?; + monotonic_tombstone_events.push(MonotonicDeleteEvent { + event_key, + new_epoch: del_iter.earliest_epoch(), + }); + } + // process the last key if !versions.is_empty() { merged_payload.push((pivot, versions)); } drop(del_iter); - let compaction_delete_events = Arc::unwrap_or_clone(compaction_delete_ranges).into_events(); - let monotonic_tombstone_events = - create_monotonic_events_from_compaction_delete_events(compaction_delete_events); Ok(SharedBufferBatch { inner: Arc::new(SharedBufferBatchInner::new_with_multi_epoch_batches( @@ -491,7 +513,7 @@ impl SharedBufferCompactRunner { self, iter: impl HummockIterator, filter_key_extractor: Arc, - del_agg: Arc, + del_iter: CompactionDeleteRangeIterator, ) -> HummockResult { let dummy_compaction_filter = DummyCompactionFilter {}; let (ssts, table_stats_map) = self @@ -499,7 +521,7 @@ impl SharedBufferCompactRunner { .compact_key_range( iter, dummy_compaction_filter, - del_agg, + del_iter, filter_key_extractor, None, None, diff --git a/src/storage/src/hummock/iterator/concat_delete_range_iterator.rs b/src/storage/src/hummock/iterator/concat_delete_range_iterator.rs index 82921da511909..8a89774e23abc 100644 --- a/src/storage/src/hummock/iterator/concat_delete_range_iterator.rs +++ b/src/storage/src/hummock/iterator/concat_delete_range_iterator.rs @@ -181,42 +181,40 @@ impl DeleteRangeIterator for ConcatDeleteRangeIterator { } #[cfg(test)] mod tests { + use std::ops::Bound; + + use bytes::Bytes; use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::MAX_EPOCH; use super::*; use crate::hummock::iterator::test_utils::mock_sstable_store; - use crate::hummock::test_utils::test_user_key; - use crate::hummock::{ - create_monotonic_events, CompactionDeleteRangesBuilder, DeleteRangeTombstone, - SstableBuilder, SstableBuilderOptions, SstableWriterOptions, - }; + use crate::hummock::test_utils::{test_user_key, CompactionDeleteRangesBuilder}; + use crate::hummock::{SstableBuilder, SstableBuilderOptions, SstableWriterOptions}; #[tokio::test] async fn test_concat_iterator() { - let mut builder = CompactionDeleteRangesBuilder::default(); let sstable_store = mock_sstable_store(); let table_id = TableId::new(0); - let data = vec![ - DeleteRangeTombstone::new_for_test(table_id, b"aaaa".to_vec(), b"dddd".to_vec(), 10), - DeleteRangeTombstone::new( - table_id, - b"bbbb".to_vec(), - true, - b"eeee".to_vec(), - false, - 12, - ), - ]; - for range in data { - builder.add_delete_events(create_monotonic_events(vec![range])); - } - - let compaction_delete_range = builder.build_for_compaction(); - let ranges1 = compaction_delete_range.get_tombstone_between( - test_user_key(b"aaaa").as_ref(), - test_user_key(b"bbbb").as_ref(), + let tombstone1 = ( + Bound::Included(Bytes::copy_from_slice(b"aaaa")), + Bound::Excluded(Bytes::copy_from_slice(b"dddd")), + ); + let tombstone2 = ( + Bound::Excluded(Bytes::copy_from_slice(b"bbbb")), + Bound::Excluded(Bytes::copy_from_slice(b"eeee")), ); + let mut range_builder = CompactionDeleteRangesBuilder::default(); + range_builder.add_delete_events(10, table_id, vec![tombstone1.clone()]); + range_builder.add_delete_events(12, table_id, vec![tombstone2.clone()]); + let compaction_delete_range = range_builder.build_for_compaction(); + let ranges1 = compaction_delete_range + .get_tombstone_between( + test_user_key(b"aaaa").as_ref(), + test_user_key(b"bbbb").as_ref(), + ) + .await + .unwrap(); assert_eq!(ranges1.len(), 2); let opts = SstableBuilderOptions::default(); let mut builder = SstableBuilder::for_test( @@ -236,8 +234,14 @@ mod tests { .create_sst_writer(2, SstableWriterOptions::default()), opts.clone(), ); + let mut range_builder = CompactionDeleteRangesBuilder::default(); + range_builder.add_delete_events(10, table_id, vec![tombstone1.clone()]); + range_builder.add_delete_events(12, table_id, vec![tombstone2.clone()]); + let compaction_delete_range = range_builder.build_for_compaction(); let ranges2 = compaction_delete_range - .get_tombstone_between(test_user_key(b"bbbb").as_ref(), test_user_key(b"").as_ref()); + .get_tombstone_between(test_user_key(b"bbbb").as_ref(), test_user_key(b"").as_ref()) + .await + .unwrap(); assert_eq!(ranges2.len(), 3); builder.add_monotonic_deletes(ranges2); let output2 = builder.finish().await.unwrap(); diff --git a/src/storage/src/hummock/iterator/delete_range_iterator.rs b/src/storage/src/hummock/iterator/delete_range_iterator.rs index 7f72a025943c1..e62c2f53d600e 100644 --- a/src/storage/src/hummock/iterator/delete_range_iterator.rs +++ b/src/storage/src/hummock/iterator/delete_range_iterator.rs @@ -231,6 +231,12 @@ pub struct ForwardMergeRangeIterator { current_epochs: BTreeSet, } +impl Default for ForwardMergeRangeIterator { + fn default() -> Self { + ForwardMergeRangeIterator::new(MAX_EPOCH) + } +} + impl ForwardMergeRangeIterator { pub fn new(read_epoch: HummockEpoch) -> Self { Self { @@ -270,6 +276,19 @@ impl ForwardMergeRangeIterator { } Ok(()) } + + pub fn earliest_delete_since(&self, epoch: HummockEpoch) -> HummockEpoch { + self.current_epochs + .range(epoch..) + .next() + .map_or(MAX_EPOCH, |ret| *ret) + } + + pub fn earliest_epoch(&self) -> HummockEpoch { + self.current_epochs + .first() + .map_or(MAX_EPOCH, |epoch| *epoch) + } } impl DeleteRangeIterator for ForwardMergeRangeIterator { diff --git a/src/storage/src/hummock/sstable/delete_range_aggregator.rs b/src/storage/src/hummock/sstable/delete_range_aggregator.rs index 953f5b16b28a5..2eac06a70a724 100644 --- a/src/storage/src/hummock/sstable/delete_range_aggregator.rs +++ b/src/storage/src/hummock/sstable/delete_range_aggregator.rs @@ -13,19 +13,15 @@ // limitations under the License. use std::cmp::Ordering; -use std::collections::{BTreeMap, BTreeSet}; +use std::collections::BTreeSet; use std::future::Future; -use std::sync::Arc; -use itertools::Itertools; use risingwave_common::util::epoch::MAX_EPOCH; use risingwave_hummock_sdk::key::{PointRange, UserKey}; use risingwave_hummock_sdk::HummockEpoch; -#[cfg(any(test, feature = "test"))] -use super::DeleteRangeTombstone; use super::MonotonicDeleteEvent; -use crate::hummock::iterator::DeleteRangeIterator; +use crate::hummock::iterator::{DeleteRangeIterator, ForwardMergeRangeIterator}; use crate::hummock::sstable_store::TableHolder; use crate::hummock::{HummockResult, Sstable}; @@ -56,11 +52,6 @@ impl Ord for SortedBoundary { } } -#[derive(Default)] -pub struct CompactionDeleteRangesBuilder { - events: Vec>, -} - #[derive(Clone)] pub(crate) struct TombstoneEnterExitEvent { pub(crate) tombstone_epoch: HummockEpoch, @@ -111,135 +102,39 @@ pub(crate) fn apply_event(epochs: &mut BTreeSet, event: &Compactio } } -#[derive(Clone, Default)] -pub struct CompactionDeleteRanges { - events: Vec, +pub struct CompactionDeleteRangeIterator { + inner: ForwardMergeRangeIterator, } -impl CompactionDeleteRangesBuilder { - pub fn add_delete_events(&mut self, data: Vec) { - self.events.push(data); - } - - /// Assume that watermark1 is 5, watermark2 is 7, watermark3 is 11, delete ranges - /// `{ [0, wmk1) in epoch1, [wmk1, wmk2) in epoch2, [wmk2, wmk3) in epoch3 }` - /// can be transformed into events below: - /// `{ <0, +epoch1> }` - #[cfg(any(test, feature = "test"))] - pub(crate) fn build_events( - delete_tombstones: &Vec, - ) -> Vec { - let tombstone_len = delete_tombstones.len(); - let mut events = Vec::with_capacity(tombstone_len * 2); - for DeleteRangeTombstone { - start_user_key, - end_user_key, - sequence, - } in delete_tombstones - { - events.push((start_user_key, 1, *sequence)); - events.push((end_user_key, 0, *sequence)); - } - events.sort(); - - let mut result = Vec::with_capacity(events.len()); - for (user_key, group) in &events.into_iter().group_by(|(user_key, _, _)| *user_key) { - let (mut exit, mut enter) = (vec![], vec![]); - for (_, op, sequence) in group { - match op { - 0 => exit.push(TombstoneEnterExitEvent { - tombstone_epoch: sequence, - }), - 1 => { - enter.push(TombstoneEnterExitEvent { - tombstone_epoch: sequence, - }); - } - _ => unreachable!(), - } - } - result.push((user_key.clone(), exit, enter)); - } - - result - } - - pub(crate) fn build_for_compaction(self) -> Arc { - let mut ret = BTreeMap::< - PointRange>, - (Vec, Vec), - >::default(); - for monotonic_deletes in self.events { - let mut last_exit_epoch = MAX_EPOCH; - for delete_event in monotonic_deletes { - if last_exit_epoch != MAX_EPOCH { - let entry = ret.entry(delete_event.event_key.clone()).or_default(); - entry.0.push(TombstoneEnterExitEvent { - tombstone_epoch: last_exit_epoch, - }); - } - if delete_event.new_epoch != MAX_EPOCH { - let entry = ret.entry(delete_event.event_key).or_default(); - entry.1.push(TombstoneEnterExitEvent { - tombstone_epoch: delete_event.new_epoch, - }); - } - last_exit_epoch = delete_event.new_epoch; - } - } - let events = ret - .into_iter() - .map(|(k, (exits, enters))| (k, exits, enters)) - .collect_vec(); - - Arc::new(CompactionDeleteRanges { events }) +impl CompactionDeleteRangeIterator { + pub fn new(inner: ForwardMergeRangeIterator) -> Self { + Self { inner } } -} -impl CompactionDeleteRanges { - pub(crate) fn iter(self: &Arc) -> CompactionDeleteRangeIterator { - CompactionDeleteRangeIterator { - events: self.clone(), - seek_idx: 0, - epochs: BTreeSet::default(), - } + pub(crate) async fn next(&mut self) -> HummockResult<()> { + self.inner.next().await } - /// the `largest_user_key` is always exclusive #[cfg(test)] - pub(crate) fn get_tombstone_between( - &self, + pub async fn get_tombstone_between( + self, smallest_user_key: UserKey<&[u8]>, largest_user_key: UserKey<&[u8]>, - ) -> Vec { - if self.events.is_empty() { - return vec![]; - } - + ) -> HummockResult> { + let mut iter = self; + iter.seek(smallest_user_key).await?; let extended_smallest_user_key = PointRange::from_user_key(smallest_user_key, false); let extended_largest_user_key = PointRange::from_user_key(largest_user_key, false); - - let mut monotonic_events = Vec::with_capacity(self.events.len()); - let mut epochs = BTreeSet::new(); - let mut idx = 0; - while idx < self.events.len() { - if self.events[idx].0.as_ref().gt(&extended_smallest_user_key) { - if let Some(epoch) = epochs.first() { - monotonic_events.push(MonotonicDeleteEvent { - event_key: extended_smallest_user_key.to_vec(), - new_epoch: *epoch, - }); - } - break; - } - apply_event(&mut epochs, &self.events[idx]); - idx += 1; + let mut monotonic_events = vec![]; + if iter.earliest_epoch() < MAX_EPOCH { + monotonic_events.push(MonotonicDeleteEvent { + event_key: extended_smallest_user_key.to_vec(), + new_epoch: iter.earliest_epoch(), + }); } - while idx < self.events.len() { - if !extended_largest_user_key.is_empty() - && self.events[idx].0.as_ref().ge(&extended_largest_user_key) - { + + while iter.is_valid() { + if !extended_largest_user_key.is_empty() && iter.key().ge(&extended_largest_user_key) { if !monotonic_events.is_empty() { monotonic_events.push(MonotonicDeleteEvent { event_key: extended_largest_user_key.to_vec(), @@ -248,13 +143,16 @@ impl CompactionDeleteRanges { } break; } - apply_event(&mut epochs, &self.events[idx]); + + let event_key = iter.key().to_vec(); + iter.next().await?; + monotonic_events.push(MonotonicDeleteEvent { - event_key: self.events[idx].0.clone(), - new_epoch: epochs.first().map_or(MAX_EPOCH, |epoch| *epoch), + new_epoch: iter.earliest_epoch(), + event_key, }); - idx += 1; } + monotonic_events.dedup_by(|a, b| { a.event_key.left_user_key.table_id == b.event_key.left_user_key.table_id && a.new_epoch == b.new_epoch @@ -263,88 +161,52 @@ impl CompactionDeleteRanges { assert_ne!(monotonic_events.first().unwrap().new_epoch, MAX_EPOCH); assert_eq!(monotonic_events.last().unwrap().new_epoch, MAX_EPOCH); } - monotonic_events - } - - pub(crate) fn into_events(self) -> Vec { - self.events - } -} - -pub(crate) struct CompactionDeleteRangeIterator { - events: Arc, - seek_idx: usize, - /// The correctness of the algorithm needs to be guaranteed by "the epoch of the - /// intervals covering each other must be different". - epochs: BTreeSet, -} - -impl CompactionDeleteRangeIterator { - fn apply(&mut self, idx: usize) { - apply_event(&mut self.epochs, &self.events.events[idx]); - } - - pub(crate) fn next(&mut self) { - self.seek_idx += 1; + Ok(monotonic_events) } /// Return the earliest range-tombstone which deletes target-key. /// Target-key must be given in order. - pub(crate) fn earliest_delete_which_can_see_key( + #[cfg(test)] + pub async fn earliest_delete_which_can_see_key( &mut self, target_user_key: UserKey<&[u8]>, epoch: HummockEpoch, - ) -> HummockEpoch { + ) -> HummockResult { let target_extended_user_key = PointRange::from_user_key(target_user_key, false); - while let Some((extended_user_key, ..)) = self.events.events.get(self.seek_idx) - && extended_user_key.as_ref().le(&target_extended_user_key) + while self.inner.is_valid() + && self + .inner + .next_extended_user_key() + .le(&target_extended_user_key) { - self.apply(self.seek_idx); - self.seek_idx += 1; + self.inner.next().await?; } - self.earliest_delete_since(epoch) - } - - pub fn update_range(&mut self) { - self.apply(self.seek_idx); + Ok(self.earliest_delete_since(epoch)) } - pub fn key(&self) -> &PointRange> { - &self.events.events[self.seek_idx].0 + pub fn key(&self) -> PointRange<&[u8]> { + self.inner.next_extended_user_key() } pub(crate) fn is_valid(&self) -> bool { - self.seek_idx < self.events.events.len() + self.inner.is_valid() } pub(crate) fn earliest_epoch(&self) -> HummockEpoch { - self.epochs.first().map_or(MAX_EPOCH, |epoch| *epoch) + self.inner.earliest_epoch() } pub(crate) fn earliest_delete_since(&self, epoch: HummockEpoch) -> HummockEpoch { - self.epochs - .range(epoch..) - .next() - .map_or(MAX_EPOCH, |ret| *ret) + self.inner.earliest_delete_since(epoch) } - pub(crate) fn seek<'a>(&'a mut self, target_user_key: UserKey<&'a [u8]>) { - let target_extended_user_key = PointRange::from_user_key(target_user_key, false); - self.seek_idx = self - .events - .events - .partition_point(|(extended_user_key, ..)| { - extended_user_key.as_ref().le(&target_extended_user_key) - }); - self.epochs.clear(); - for idx in 0..self.seek_idx { - self.apply(idx); - } + /// seek to the first key which larger than `target_user_key`. + pub async fn seek<'a>(&'a mut self, target_user_key: UserKey<&'a [u8]>) -> HummockResult<()> { + self.inner.seek(target_user_key).await } - pub(crate) fn rewind(&mut self) { - self.seek_idx = 0; - self.epochs.clear(); + pub async fn rewind(&mut self) -> HummockResult<()> { + self.inner.rewind().await } } @@ -442,147 +304,195 @@ pub fn get_min_delete_range_epoch_from_sstable( #[cfg(test)] mod tests { + use std::ops::Bound; + + use bytes::Bytes; use risingwave_common::catalog::TableId; - use risingwave_hummock_sdk::key::TableKey; use super::*; - use crate::hummock::create_monotonic_events; use crate::hummock::iterator::test_utils::{ gen_iterator_test_sstable_with_range_tombstones, iterator_test_user_key_of, mock_sstable_store, }; - use crate::hummock::test_utils::test_user_key; + use crate::hummock::test_utils::{test_user_key, CompactionDeleteRangesBuilder}; - #[test] - pub fn test_compaction_delete_range_iterator() { + #[tokio::test] + pub async fn test_compaction_delete_range_iterator() { let mut builder = CompactionDeleteRangesBuilder::default(); let table_id = TableId::default(); - let data = vec![ - DeleteRangeTombstone::new_for_test( - table_id, - b"aaaaaa".to_vec(), - b"bbbccc".to_vec(), - 12, - ), - DeleteRangeTombstone::new_for_test(table_id, b"aaaaaa".to_vec(), b"bbbddd".to_vec(), 9), - DeleteRangeTombstone::new_for_test(table_id, b"bbbfff".to_vec(), b"ffffff".to_vec(), 9), - DeleteRangeTombstone::new_for_test(table_id, b"gggggg".to_vec(), b"hhhhhh".to_vec(), 9), - DeleteRangeTombstone::new( - table_id, - b"bbbeee".to_vec(), - true, - b"eeeeee".to_vec(), - true, - 8, - ), - DeleteRangeTombstone::new_for_test( - table_id, - b"bbbaab".to_vec(), - b"bbbdddf".to_vec(), - 6, - ), - DeleteRangeTombstone { - start_user_key: PointRange::from_user_key( - UserKey::new(table_id, TableKey(b"hhhhhh".to_vec())), - true, + builder.add_delete_events( + 9, + table_id, + vec![ + ( + Bound::Included(Bytes::copy_from_slice(b"aaaaaa")), + Bound::Excluded(Bytes::copy_from_slice(b"bbbddd")), ), - end_user_key: PointRange::from_user_key( - UserKey::new(TableId::new(table_id.table_id() + 1), TableKey::default()), - false, + ( + Bound::Included(Bytes::copy_from_slice(b"bbbfff")), + Bound::Excluded(Bytes::copy_from_slice(b"ffffff")), ), - sequence: 7, - }, - ]; - for range in data { - builder.add_delete_events(create_monotonic_events(vec![range])); - } - let compaction_delete_ranges = builder.build_for_compaction(); - let mut iter = compaction_delete_ranges.iter(); + ( + Bound::Included(Bytes::copy_from_slice(b"gggggg")), + Bound::Excluded(Bytes::copy_from_slice(b"hhhhhh")), + ), + ], + ); + builder.add_delete_events( + 12, + table_id, + vec![( + Bound::Included(Bytes::copy_from_slice(b"aaaaaa")), + Bound::Excluded(Bytes::copy_from_slice(b"bbbccc")), + )], + ); + builder.add_delete_events( + 8, + table_id, + vec![( + Bound::Excluded(Bytes::copy_from_slice(b"bbbeee")), + Bound::Included(Bytes::copy_from_slice(b"eeeeee")), + )], + ); + builder.add_delete_events( + 6, + table_id, + vec![( + Bound::Included(Bytes::copy_from_slice(b"bbbaab")), + Bound::Excluded(Bytes::copy_from_slice(b"bbbdddf")), + )], + ); + builder.add_delete_events( + 7, + table_id, + vec![( + Bound::Excluded(Bytes::copy_from_slice(b"hhhhhh")), + Bound::Unbounded, + )], + ); + let mut iter = builder.build_for_compaction(); + iter.rewind().await.unwrap(); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"bbb").as_ref(), 13), - MAX_EPOCH + iter.earliest_delete_which_can_see_key(test_user_key(b"bbb").as_ref(), 13) + .await + .unwrap(), + MAX_EPOCH, ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"bbb").as_ref(), 11), + iter.earliest_delete_which_can_see_key(test_user_key(b"bbb").as_ref(), 11) + .await + .unwrap(), 12 ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"bbb").as_ref(), 8), + iter.earliest_delete_which_can_see_key(test_user_key(b"bbb").as_ref(), 8) + .await + .unwrap(), 9 ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"bbbaaa").as_ref(), 8), + iter.earliest_delete_which_can_see_key(test_user_key(b"bbbaaa").as_ref(), 8) + .await + .unwrap(), 9 ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"bbbccd").as_ref(), 8), + iter.earliest_delete_which_can_see_key(test_user_key(b"bbbccd").as_ref(), 8) + .await + .unwrap(), 9 ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"bbbddd").as_ref(), 8), - MAX_EPOCH + iter.earliest_delete_which_can_see_key(test_user_key(b"bbbddd").as_ref(), 8) + .await + .unwrap(), + MAX_EPOCH, ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"bbbeee").as_ref(), 8), - MAX_EPOCH + iter.earliest_delete_which_can_see_key(test_user_key(b"bbbeee").as_ref(), 8) + .await + .unwrap(), + MAX_EPOCH, ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"bbbeef").as_ref(), 10), - MAX_EPOCH + iter.earliest_delete_which_can_see_key(test_user_key(b"bbbeef").as_ref(), 10) + .await + .unwrap(), + MAX_EPOCH, ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"eeeeee").as_ref(), 8), + iter.earliest_delete_which_can_see_key(test_user_key(b"eeeeee").as_ref(), 8) + .await + .unwrap(), 8 ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"gggggg").as_ref(), 8), + iter.earliest_delete_which_can_see_key(test_user_key(b"gggggg").as_ref(), 8) + .await + .unwrap(), 9 ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"hhhhhh").as_ref(), 6), - MAX_EPOCH + iter.earliest_delete_which_can_see_key(test_user_key(b"hhhhhh").as_ref(), 6) + .await + .unwrap(), + MAX_EPOCH, ); assert_eq!( - iter.earliest_delete_which_can_see_key(test_user_key(b"iiiiii").as_ref(), 6), + iter.earliest_delete_which_can_see_key(test_user_key(b"iiiiii").as_ref(), 6) + .await + .unwrap(), 7 ); } - #[test] - pub fn test_delete_range_split() { + #[tokio::test] + pub async fn test_delete_range_split() { let table_id = TableId::default(); let mut builder = CompactionDeleteRangesBuilder::default(); - let data = vec![ - DeleteRangeTombstone::new_for_test(table_id, b"aaaa".to_vec(), b"cccc".to_vec(), 13), - DeleteRangeTombstone::new( - table_id, - b"cccc".to_vec(), - true, - b"dddd".to_vec(), - false, - 10, - ), - DeleteRangeTombstone::new( - table_id, - b"cccc".to_vec(), - false, - b"eeee".to_vec(), - true, - 12, - ), - DeleteRangeTombstone::new(table_id, b"eeee".to_vec(), true, b"ffff".to_vec(), true, 15), - ]; - for range in data { - builder.add_delete_events(create_monotonic_events(vec![range])); - } - let compaction_delete_range = builder.build_for_compaction(); - let split_ranges = compaction_delete_range.get_tombstone_between( - test_user_key(b"bbbb").as_ref(), - test_user_key(b"eeeeee").as_ref(), + builder.add_delete_events( + 13, + table_id, + vec![( + Bound::Included(Bytes::copy_from_slice(b"aaaa")), + Bound::Excluded(Bytes::copy_from_slice(b"cccc")), + )], ); + builder.add_delete_events( + 10, + table_id, + vec![( + Bound::Excluded(Bytes::copy_from_slice(b"cccc")), + Bound::Excluded(Bytes::copy_from_slice(b"dddd")), + )], + ); + builder.add_delete_events( + 12, + table_id, + vec![( + Bound::Included(Bytes::copy_from_slice(b"cccc")), + Bound::Included(Bytes::copy_from_slice(b"eeee")), + )], + ); + builder.add_delete_events( + 15, + table_id, + vec![( + Bound::Excluded(Bytes::copy_from_slice(b"eeee")), + Bound::Excluded(Bytes::copy_from_slice(b"ffff")), + )], + ); + let compaction_delete_range = builder.build_for_compaction(); + let split_ranges = compaction_delete_range + .get_tombstone_between( + test_user_key(b"bbbb").as_ref(), + test_user_key(b"eeeeee").as_ref(), + ) + .await + .unwrap(); assert_eq!(6, split_ranges.len()); assert_eq!( PointRange::from_user_key(test_user_key(b"bbbb"), false), diff --git a/src/storage/src/hummock/sstable/mod.rs b/src/storage/src/hummock/sstable/mod.rs index 1c6b5aeedea5a..039bf613763b1 100644 --- a/src/storage/src/hummock/sstable/mod.rs +++ b/src/storage/src/hummock/sstable/mod.rs @@ -55,10 +55,11 @@ mod sstable_object_id_manager; mod utils; pub use delete_range_aggregator::{ - get_min_delete_range_epoch_from_sstable, CompactionDeleteRanges, CompactionDeleteRangesBuilder, + get_min_delete_range_epoch_from_sstable, CompactionDeleteRangeIterator, SstableDeleteRangeIterator, }; pub use filter::FilterBuilder; +use itertools::Itertools; pub use sstable_object_id_manager::*; pub use utils::CompressionAlgorithm; use utils::{get_length_prefixed_slice, put_length_prefixed_slice}; @@ -67,6 +68,7 @@ use xxhash_rust::{xxh32, xxh64}; use self::delete_range_aggregator::{apply_event, CompactionDeleteRangeEvent}; use self::utils::{xxhash64_checksum, xxhash64_verify}; use super::{HummockError, HummockResult}; +use crate::hummock::sstable::delete_range_aggregator::TombstoneEnterExitEvent; use crate::hummock::CachePolicy; use crate::store::ReadOptions; @@ -200,7 +202,8 @@ impl MonotonicDeleteEvent { } } -pub(crate) fn create_monotonic_events_from_compaction_delete_events( +#[cfg(any(test, feature = "test"))] +fn create_monotonic_events_from_compaction_delete_events( compaction_delete_range_events: Vec, ) -> Vec { let mut epochs = BTreeSet::new(); @@ -219,12 +222,54 @@ pub(crate) fn create_monotonic_events_from_compaction_delete_events( monotonic_tombstone_events } +/// Assume that watermark1 is 5, watermark2 is 7, watermark3 is 11, delete ranges +/// `{ [0, wmk1) in epoch1, [wmk1, wmk2) in epoch2, [wmk2, wmk3) in epoch3 }` +/// can be transformed into events below: +/// `{ <0, +epoch1> }` +#[cfg(any(test, feature = "test"))] +fn build_events(delete_tombstones: &Vec) -> Vec { + let tombstone_len = delete_tombstones.len(); + let mut events = Vec::with_capacity(tombstone_len * 2); + for DeleteRangeTombstone { + start_user_key, + end_user_key, + sequence, + } in delete_tombstones + { + events.push((start_user_key, 1, *sequence)); + events.push((end_user_key, 0, *sequence)); + } + events.sort(); + + let mut result = Vec::with_capacity(events.len()); + for (user_key, group) in &events.into_iter().group_by(|(user_key, _, _)| *user_key) { + let (mut exit, mut enter) = (vec![], vec![]); + for (_, op, sequence) in group { + match op { + 0 => exit.push(TombstoneEnterExitEvent { + tombstone_epoch: sequence, + }), + 1 => { + enter.push(TombstoneEnterExitEvent { + tombstone_epoch: sequence, + }); + } + _ => unreachable!(), + } + } + result.push((user_key.clone(), exit, enter)); + } + + result +} + #[cfg(any(test, feature = "test"))] pub(crate) fn create_monotonic_events( mut delete_range_tombstones: Vec, ) -> Vec { delete_range_tombstones.sort(); - let events = CompactionDeleteRangesBuilder::build_events(&delete_range_tombstones); + let events = build_events(&delete_range_tombstones); create_monotonic_events_from_compaction_delete_events(events) } diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index b6d181b48894a..7f2833e707c9d 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -445,6 +445,8 @@ impl TableBuilderFactory for LocalTableBuilderFactory { #[cfg(test)] mod tests { + use std::ops::Bound; + use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; @@ -453,11 +455,10 @@ mod tests { use super::*; use crate::hummock::iterator::test_utils::mock_sstable_store; - use crate::hummock::test_utils::{default_builder_opt_for_test, test_key_of, test_user_key_of}; - use crate::hummock::{ - create_monotonic_events, CompactionDeleteRangesBuilder, DeleteRangeTombstone, - SstableBuilderOptions, DEFAULT_RESTART_INTERVAL, + use crate::hummock::test_utils::{ + default_builder_opt_for_test, test_key_of, test_user_key_of, CompactionDeleteRangesBuilder, }; + use crate::hummock::{SstableBuilderOptions, DEFAULT_RESTART_INTERVAL}; #[tokio::test] async fn test_empty() { @@ -571,35 +572,32 @@ mod tests { let opts = default_builder_opt_for_test(); let table_id = TableId::default(); let mut builder = CompactionDeleteRangesBuilder::default(); - let events = create_monotonic_events(vec![ - DeleteRangeTombstone::new( - table_id, - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"k"] - .concat() - .to_vec(), - false, - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"kkk"] - .concat() - .to_vec(), - false, - 100, - ), - DeleteRangeTombstone::new( - table_id, - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"aaa"] - .concat() - .to_vec(), - false, - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"ddd"] - .concat() - .to_vec(), - false, - 200, - ), - ]); - builder.add_delete_events(events); - let agg = builder.build_for_compaction(); - let mut del_iter = agg.iter(); + builder.add_delete_events( + 100, + table_id, + vec![( + Bound::Included(Bytes::copy_from_slice( + &[VirtualNode::ZERO.to_be_bytes().as_slice(), b"k"].concat(), + )), + Bound::Excluded(Bytes::copy_from_slice( + &[VirtualNode::ZERO.to_be_bytes().as_slice(), b"kkk"].concat(), + )), + )], + ); + builder.add_delete_events( + 200, + table_id, + vec![( + Bound::Included(Bytes::copy_from_slice( + &[VirtualNode::ZERO.to_be_bytes().as_slice(), b"aaa"].concat(), + )), + Bound::Excluded(Bytes::copy_from_slice( + &[VirtualNode::ZERO.to_be_bytes().as_slice(), b"ddd"].concat(), + )), + )], + ); + let mut del_iter = builder.build_for_compaction(); + del_iter.rewind().await.unwrap(); let mut builder = CapacitySplitTableBuilder::new( LocalTableBuilderFactory::new(1001, mock_sstable_store(), opts), Arc::new(CompactorMetrics::unused()), @@ -615,30 +613,30 @@ mod tests { ); let target_extended_user_key = PointRange::from_user_key(full_key.user_key.as_ref(), false); while del_iter.is_valid() && del_iter.key().as_ref().le(&target_extended_user_key) { - del_iter.update_range(); + let event_key = del_iter.key().to_vec(); + del_iter.next().await.unwrap(); builder .add_monotonic_delete(MonotonicDeleteEvent { - event_key: del_iter.key().clone(), new_epoch: del_iter.earliest_epoch(), + event_key, }) .await .unwrap(); - del_iter.next(); } builder .add_full_key(full_key.to_ref(), HummockValue::put(b"v"), false) .await .unwrap(); while del_iter.is_valid() { - del_iter.update_range(); + let event_key = del_iter.key().to_vec(); + del_iter.next().await.unwrap(); builder .add_monotonic_delete(MonotonicDeleteEvent { - event_key: del_iter.key().clone(), + event_key, new_epoch: del_iter.earliest_epoch(), }) .await .unwrap(); - del_iter.next(); } let mut sst_infos = builder.finish().await.unwrap(); let key_range = sst_infos @@ -681,12 +679,23 @@ mod tests { }; let table_id = TableId::new(1); let mut builder = CompactionDeleteRangesBuilder::default(); - builder.add_delete_events(create_monotonic_events(vec![ - DeleteRangeTombstone::new_for_test(table_id, b"k".to_vec(), b"kkk".to_vec(), 100), - DeleteRangeTombstone::new_for_test(table_id, b"aaa".to_vec(), b"ddd".to_vec(), 200), - ])); - let agg = builder.build_for_compaction(); - let mut del_iter = agg.iter(); + builder.add_delete_events( + 100, + table_id, + vec![( + Bound::Included(Bytes::copy_from_slice(b"k")), + (Bound::Excluded(Bytes::copy_from_slice(b"kkk"))), + )], + ); + builder.add_delete_events( + 200, + table_id, + vec![( + Bound::Included(Bytes::copy_from_slice(b"aaa")), + (Bound::Excluded(Bytes::copy_from_slice(b"ddd"))), + )], + ); + let mut del_iter = builder.build_for_compaction(); let mut builder = CapacitySplitTableBuilder::new( LocalTableBuilderFactory::new(1001, mock_sstable_store(), opts), Arc::new(CompactorMetrics::unused()), @@ -695,17 +704,18 @@ mod tests { false, 0, ); + del_iter.rewind().await.unwrap(); assert_eq!(del_iter.earliest_epoch(), MAX_EPOCH); while del_iter.is_valid() { - del_iter.update_range(); + let event_key = del_iter.key().to_vec(); + del_iter.next().await.unwrap(); builder .add_monotonic_delete(MonotonicDeleteEvent { - event_key: del_iter.key().clone(), new_epoch: del_iter.earliest_epoch(), + event_key, }) .await .unwrap(); - del_iter.next(); } let results = builder.finish().await.unwrap(); diff --git a/src/storage/src/hummock/test_utils.rs b/src/storage/src/hummock/test_utils.rs index a5a00413c8d20..91093804ca5d8 100644 --- a/src/storage/src/hummock/test_utils.rs +++ b/src/storage/src/hummock/test_utils.rs @@ -28,16 +28,17 @@ use risingwave_pb::hummock::{KeyRange, SstableInfo}; use super::iterator::test_utils::iterator_test_table_key_of; use super::{ - create_monotonic_events, HummockResult, InMemWriter, SstableMeta, SstableWriterOptions, - DEFAULT_RESTART_INTERVAL, + HummockResult, InMemWriter, SstableMeta, SstableWriterOptions, DEFAULT_RESTART_INTERVAL, }; use crate::error::StorageResult; use crate::filter_key_extractor::{FilterKeyExtractorImpl, FullKeyFilterKeyExtractor}; +use crate::hummock::iterator::ForwardMergeRangeIterator; use crate::hummock::shared_buffer::shared_buffer_batch::SharedBufferBatch; use crate::hummock::value::HummockValue; use crate::hummock::{ - BlockedXor16FilterBuilder, CachePolicy, DeleteRangeTombstone, FilterBuilder, LruCache, Sstable, - SstableBuilder, SstableBuilderOptions, SstableStoreRef, SstableWriter, Xor16FilterBuilder, + create_monotonic_events, BlockedXor16FilterBuilder, CachePolicy, CompactionDeleteRangeIterator, + DeleteRangeTombstone, FilterBuilder, LruCache, Sstable, SstableBuilder, SstableBuilderOptions, + SstableStoreRef, SstableWriter, Xor16FilterBuilder, }; use crate::monitor::StoreLocalStatistic; use crate::opts::StorageOpts; @@ -392,3 +393,34 @@ pub async fn count_stream(s: impl Stream> + Send) -> pub fn create_small_table_cache() -> Arc>> { Arc::new(LruCache::new(1, 4, 0)) } + +#[derive(Default)] +pub struct CompactionDeleteRangesBuilder { + iter: ForwardMergeRangeIterator, +} + +impl CompactionDeleteRangesBuilder { + pub fn add_delete_events( + &mut self, + epoch: HummockEpoch, + table_id: TableId, + delete_ranges: Vec<(Bound, Bound)>, + ) { + let size = SharedBufferBatch::measure_delete_range_size(&delete_ranges); + let batch = SharedBufferBatch::build_shared_buffer_batch( + epoch, + 0, + vec![], + size, + delete_ranges, + table_id, + None, + None, + ); + self.iter.add_batch_iter(batch.delete_range_iter()); + } + + pub fn build_for_compaction(self) -> CompactionDeleteRangeIterator { + CompactionDeleteRangeIterator::new(self.iter) + } +} From c64a0a9a9bfc6590c4f47486a058616477236dc4 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Mon, 13 Nov 2023 17:11:49 +0800 Subject: [PATCH 46/77] refactor(stream): use `ExecutorParams::info` in over window related executors (#13383) Signed-off-by: Richard Chien --- src/stream/src/executor/over_window/eowc.rs | 25 ++++------------ .../src/executor/over_window/general.rs | 29 +++++-------------- src/stream/src/executor/top_n/group_top_n.rs | 4 +-- .../executor/top_n/group_top_n_appendonly.rs | 4 +-- .../src/executor/top_n/top_n_appendonly.rs | 4 +-- src/stream/src/executor/top_n/top_n_plain.rs | 2 +- src/stream/src/executor/top_n/utils.rs | 21 ++------------ src/stream/src/from_proto/eowc_over_window.rs | 7 +++-- src/stream/src/from_proto/group_top_n.rs | 8 +---- src/stream/src/from_proto/over_window.rs | 9 ++++-- src/stream/src/from_proto/top_n.rs | 8 +---- .../integration_tests/eowc_over_window.rs | 22 +++++++++++--- .../tests/integration_tests/over_window.rs | 20 ++++++++++--- 13 files changed, 70 insertions(+), 93 deletions(-) diff --git a/src/stream/src/executor/over_window/eowc.rs b/src/stream/src/executor/over_window/eowc.rs index fa20e3b49d970..35d87af1a8afe 100644 --- a/src/stream/src/executor/over_window/eowc.rs +++ b/src/stream/src/executor/over_window/eowc.rs @@ -20,7 +20,7 @@ use futures_async_stream::{for_await, try_stream}; use itertools::Itertools; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{ArrayRef, Op, StreamChunk}; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::Schema; use risingwave_common::estimate_size::collections::VecDeque; use risingwave_common::estimate_size::EstimateSize; use risingwave_common::row::{OwnedRow, Row, RowExt}; @@ -40,7 +40,7 @@ use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::StateTable; use crate::executor::{ expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndices, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, }; use crate::task::AtomicU64Ref; @@ -136,11 +136,10 @@ impl Executor for EowcOverWindowExecutor { } pub struct EowcOverWindowExecutorArgs { - pub input: BoxedExecutor, - pub actor_ctx: ActorContextRef, - pub pk_indices: PkIndices, - pub executor_id: u64, + pub info: ExecutorInfo, + + pub input: BoxedExecutor, pub calls: Vec, pub partition_key_indices: Vec, @@ -153,23 +152,11 @@ impl EowcOverWindowExecutor { pub fn new(args: EowcOverWindowExecutorArgs) -> Self { let input_info = args.input.info(); - let schema = { - let mut schema = input_info.schema.clone(); - args.calls.iter().for_each(|call| { - schema.fields.push(Field::unnamed(call.return_type.clone())); - }); - schema - }; - Self { input: args.input, inner: ExecutorInner { actor_ctx: args.actor_ctx, - info: ExecutorInfo { - schema, - pk_indices: args.pk_indices, - identity: format!("EowcOverWindowExecutor {:X}", args.executor_id), - }, + info: args.info, calls: args.calls, input_pk_indices: input_info.pk_indices, partition_key_indices: args.partition_key_indices, diff --git a/src/stream/src/executor/over_window/general.rs b/src/stream/src/executor/over_window/general.rs index 4b105e2994f65..fe2d828a4ad0e 100644 --- a/src/stream/src/executor/over_window/general.rs +++ b/src/stream/src/executor/over_window/general.rs @@ -22,7 +22,6 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, RowRef, StreamChunk}; -use risingwave_common::catalog::Field; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::session_config::OverWindowCachePolicy as CachePolicy; use risingwave_common::types::{DataType, DefaultOrdered}; @@ -47,7 +46,7 @@ use crate::executor::over_window::delta_btree_map::PositionType; use crate::executor::test_utils::prelude::StateTable; use crate::executor::{ expect_first_barrier, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, - PkIndices, StreamExecutorError, StreamExecutorResult, + StreamExecutorError, StreamExecutorResult, }; use crate::task::AtomicU64Ref; @@ -143,11 +142,10 @@ impl ExecutorInner { } pub struct OverWindowExecutorArgs { - pub input: BoxedExecutor, - pub actor_ctx: ActorContextRef, - pub pk_indices: PkIndices, - pub executor_id: u64, + pub info: ExecutorInfo, + + pub input: BoxedExecutor, pub calls: Vec, pub partition_key_indices: Vec, @@ -165,14 +163,7 @@ pub struct OverWindowExecutorArgs { impl OverWindowExecutor { pub fn new(args: OverWindowExecutorArgs) -> Self { let input_info = args.input.info(); - - let schema = { - let mut schema = input_info.schema.clone(); - args.calls.iter().for_each(|call| { - schema.fields.push(Field::unnamed(call.return_type.clone())); - }); - schema - }; + let input_schema = &input_info.schema; let has_unbounded_frame = args.calls.iter().any(|call| call.frame.is_unbounded()); let cache_policy = if has_unbounded_frame { @@ -186,25 +177,21 @@ impl OverWindowExecutor { let order_key_data_types = args .order_key_indices .iter() - .map(|i| schema.fields()[*i].data_type.clone()) + .map(|i| input_schema.fields()[*i].data_type.clone()) .collect(); Self { input: args.input, inner: ExecutorInner { actor_ctx: args.actor_ctx, - info: ExecutorInfo { - schema, - pk_indices: args.pk_indices, - identity: format!("OverWindowExecutor {:X}", args.executor_id), - }, + info: args.info, calls: args.calls, partition_key_indices: args.partition_key_indices, order_key_indices: args.order_key_indices, order_key_data_types, order_key_order_types: args.order_key_order_types, input_pk_indices: input_info.pk_indices, - input_schema_len: input_info.schema.len(), + input_schema_len: input_schema.len(), state_table: args.state_table, watermark_epoch: args.watermark_epoch, metrics: args.metrics, diff --git a/src/stream/src/executor/top_n/group_top_n.rs b/src/stream/src/executor/top_n/group_top_n.rs index a162527bcc830..204d93a2558ae 100644 --- a/src/stream/src/executor/top_n/group_top_n.rs +++ b/src/stream/src/executor/top_n/group_top_n.rs @@ -191,7 +191,7 @@ where .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .inc(); let mut topn_cache = - TopNCache::new(self.offset, self.limit, self.schema().data_types()); + TopNCache::new(self.offset, self.limit, self.info().schema.data_types()); self.managed_state .init_topn_cache(Some(group_key), &mut topn_cache) .await?; @@ -227,7 +227,7 @@ where .group_top_n_cached_entry_count .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(self.caches.len() as i64); - generate_output(res_rows, res_ops, self.schema()) + generate_output(res_rows, res_ops, &self.info().schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { diff --git a/src/stream/src/executor/top_n/group_top_n_appendonly.rs b/src/stream/src/executor/top_n/group_top_n_appendonly.rs index 7204acae0d3af..bf8cbdb0a6134 100644 --- a/src/stream/src/executor/top_n/group_top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/group_top_n_appendonly.rs @@ -161,7 +161,7 @@ where let mut res_rows = Vec::with_capacity(self.limit); let keys = K::build(&self.group_by, chunk.data_chunk())?; - let data_types = self.schema().data_types(); + let data_types = self.info().schema.data_types(); let row_deserializer = RowDeserializer::new(data_types.clone()); let table_id_str = self.managed_state.state_table.table_id().to_string(); let actor_id_str = self.ctx.id.to_string(); @@ -211,7 +211,7 @@ where .group_top_n_appendonly_cached_entry_count .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(self.caches.len() as i64); - generate_output(res_rows, res_ops, self.schema()) + generate_output(res_rows, res_ops, &self.info().schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { diff --git a/src/stream/src/executor/top_n/top_n_appendonly.rs b/src/stream/src/executor/top_n/top_n_appendonly.rs index 499ba97ee3084..6392b0ac491fe 100644 --- a/src/stream/src/executor/top_n/top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/top_n_appendonly.rs @@ -111,7 +111,7 @@ where async fn apply_chunk(&mut self, chunk: StreamChunk) -> StreamExecutorResult { let mut res_ops = Vec::with_capacity(self.cache.limit); let mut res_rows = Vec::with_capacity(self.cache.limit); - let data_types = self.schema().data_types(); + let data_types = self.info().schema.data_types(); let row_deserializer = RowDeserializer::new(data_types); // apply the chunk to state table for (op, row_ref) in chunk.rows() { @@ -128,7 +128,7 @@ where )?; } - generate_output(res_rows, res_ops, self.schema()) + generate_output(res_rows, res_ops, &self.info().schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { diff --git a/src/stream/src/executor/top_n/top_n_plain.rs b/src/stream/src/executor/top_n/top_n_plain.rs index 933cc9a31d7c2..fe1a078e2d5a1 100644 --- a/src/stream/src/executor/top_n/top_n_plain.rs +++ b/src/stream/src/executor/top_n/top_n_plain.rs @@ -162,7 +162,7 @@ where } } } - generate_output(res_rows, res_ops, self.schema()) + generate_output(res_rows, res_ops, &self.info().schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { diff --git a/src/stream/src/executor/top_n/utils.rs b/src/stream/src/executor/top_n/utils.rs index 7c04485673c5e..9ea2dc141653f 100644 --- a/src/stream/src/executor/top_n/utils.rs +++ b/src/stream/src/executor/top_n/utils.rs @@ -49,21 +49,6 @@ pub trait TopNExecutorBase: Send + 'static { fn info(&self) -> &ExecutorInfo; - /// See [`Executor::schema`]. - fn schema(&self) -> &Schema { - &self.info().schema - } - - /// See [`Executor::pk_indices`]. - fn pk_indices(&self) -> PkIndicesRef<'_> { - self.info().pk_indices.as_ref() - } - - /// See [`Executor::identity`]. - fn identity(&self) -> &str { - &self.info().identity - } - /// Update the vnode bitmap for the state table and manipulate the cache if necessary, only used /// by Group Top-N since it's distributed. fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) { @@ -98,15 +83,15 @@ where } fn schema(&self) -> &Schema { - self.inner.schema() + &self.inner.info().schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - self.inner.pk_indices() + &self.inner.info().pk_indices } fn identity(&self) -> &str { - self.inner.identity() + &self.inner.info().identity } fn info(&self) -> ExecutorInfo { diff --git a/src/stream/src/from_proto/eowc_over_window.rs b/src/stream/src/from_proto/eowc_over_window.rs index a6f9615d1a2a5..f2e750ea3e2d3 100644 --- a/src/stream/src/from_proto/eowc_over_window.rs +++ b/src/stream/src/from_proto/eowc_over_window.rs @@ -58,10 +58,11 @@ impl ExecutorBuilder for EowcOverWindowExecutorBuilder { StateTable::from_table_catalog_inconsistent_op(node.get_state_table()?, store, vnodes) .await; Ok(EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs { - input, actor_ctx: params.actor_context, - pk_indices: params.pk_indices, - executor_id: params.executor_id, + info: params.info, + + input, + calls, partition_key_indices, order_key_index, diff --git a/src/stream/src/from_proto/group_top_n.rs b/src/stream/src/from_proto/group_top_n.rs index aed722e41132d..0f36ddc1591f9 100644 --- a/src/stream/src/from_proto/group_top_n.rs +++ b/src/stream/src/from_proto/group_top_n.rs @@ -59,16 +59,10 @@ impl ExecutorBuilder for GroupTopNExecutorBuilder ExecutorBuilder for TopNExecutorBuilder { Ok($excutor::<_, $with_ties>::new( input, params.actor_context, - info, + params.info, storage_key, (node.offset as usize, node.limit as usize), order_by, diff --git a/src/stream/tests/integration_tests/eowc_over_window.rs b/src/stream/tests/integration_tests/eowc_over_window.rs index 7334654d8dd50..cd8d4fce6452a 100644 --- a/src/stream/tests/integration_tests/eowc_over_window.rs +++ b/src/stream/tests/integration_tests/eowc_over_window.rs @@ -14,7 +14,9 @@ use risingwave_expr::aggregate::{AggArgs, AggKind}; use risingwave_expr::window_function::{Frame, FrameBound, WindowFuncCall, WindowFuncKind}; -use risingwave_stream::executor::{EowcOverWindowExecutor, EowcOverWindowExecutorArgs}; +use risingwave_stream::executor::{ + EowcOverWindowExecutor, EowcOverWindowExecutorArgs, ExecutorInfo, +}; use crate::prelude::*; @@ -45,6 +47,13 @@ async fn create_executor( OrderType::ascending(), ]; + let output_schema = { + let mut fields = input_schema.fields.clone(); + calls.iter().for_each(|call| { + fields.push(Field::unnamed(call.return_type.clone())); + }); + Schema { fields } + }; let output_pk_indices = vec![2]; let state_table = StateTable::new_without_distribution_inconsistent_op( @@ -58,10 +67,15 @@ async fn create_executor( let (tx, source) = MockSource::channel(input_schema, input_pk_indices.clone()); let executor = EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs { - input: source.boxed(), actor_ctx: ActorContext::create(123), - pk_indices: output_pk_indices, - executor_id: 1, + info: ExecutorInfo { + schema: output_schema, + pk_indices: output_pk_indices, + identity: "EowcOverWindowExecutor".to_string(), + }, + + input: source.boxed(), + calls, partition_key_indices, order_key_index, diff --git a/src/stream/tests/integration_tests/over_window.rs b/src/stream/tests/integration_tests/over_window.rs index f59a92df492f1..167528c3cb21b 100644 --- a/src/stream/tests/integration_tests/over_window.rs +++ b/src/stream/tests/integration_tests/over_window.rs @@ -18,7 +18,7 @@ use risingwave_expr::window_function::{ Frame, FrameBound, FrameExclusion, WindowFuncCall, WindowFuncKind, }; use risingwave_stream::executor::monitor::StreamingMetrics; -use risingwave_stream::executor::{OverWindowExecutor, OverWindowExecutorArgs}; +use risingwave_stream::executor::{ExecutorInfo, OverWindowExecutor, OverWindowExecutorArgs}; use crate::prelude::*; @@ -56,6 +56,13 @@ async fn create_executor( OrderType::ascending(), ]; + let output_schema = { + let mut fields = input_schema.fields.clone(); + calls.iter().for_each(|call| { + fields.push(Field::unnamed(call.return_type.clone())); + }); + Schema { fields } + }; let output_pk_indices = vec![2]; let state_table = StateTable::new_without_distribution( @@ -69,10 +76,15 @@ async fn create_executor( let (tx, source) = MockSource::channel(input_schema, input_pk_indices.clone()); let executor = OverWindowExecutor::new(OverWindowExecutorArgs { - input: source.boxed(), actor_ctx: ActorContext::create(123), - pk_indices: output_pk_indices, - executor_id: 1, + info: ExecutorInfo { + schema: output_schema, + pk_indices: output_pk_indices, + identity: "OverWindowExecutor".to_string(), + }, + + input: source.boxed(), + calls, partition_key_indices, order_key_indices, From df1a260ea8f6140646d4bc659d8f7e26516d5911 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Nov 2023 17:12:20 +0800 Subject: [PATCH 47/77] chore(deps-dev): Bump pyarrow from 12.0.1 to 14.0.1 in /integration_tests/iceberg-sink2/python (#13352) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Renjie Liu --- .../iceberg-sink2/python/poetry.lock | 155 ++++++++++++++---- 1 file changed, 122 insertions(+), 33 deletions(-) diff --git a/integration_tests/iceberg-sink2/python/poetry.lock b/integration_tests/iceberg-sink2/python/poetry.lock index c495f2c402f7f..32a6a312ef3cc 100644 --- a/integration_tests/iceberg-sink2/python/poetry.lock +++ b/integration_tests/iceberg-sink2/python/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 1.5.1 and should not be changed by hand. +# This file is automatically @generated by Poetry 1.6.1 and should not be changed by hand. [[package]] name = "googleapis-common-protos" @@ -159,10 +159,7 @@ files = [ ] [package.dependencies] -numpy = [ - {version = ">=1.21.0", markers = "python_version >= \"3.10\""}, - {version = ">=1.23.2", markers = "python_version >= \"3.11\""}, -] +numpy = {version = ">=1.23.2", markers = "python_version >= \"3.11\""} python-dateutil = ">=2.8.2" pytz = ">=2020.1" tzdata = ">=2022.1" @@ -212,6 +209,87 @@ files = [ {file = "protobuf-4.24.0.tar.gz", hash = "sha256:5d0ceb9de6e08311832169e601d1fc71bd8e8c779f3ee38a97a78554945ecb85"}, ] +[[package]] +name = "psycopg2-binary" +version = "2.9.9" +description = "psycopg2 - Python-PostgreSQL Database Adapter" +optional = false +python-versions = ">=3.7" +files = [ + {file = "psycopg2-binary-2.9.9.tar.gz", hash = "sha256:7f01846810177d829c7692f1f5ada8096762d9172af1b1a28d4ab5b77c923c1c"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c2470da5418b76232f02a2fcd2229537bb2d5a7096674ce61859c3229f2eb202"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c6af2a6d4b7ee9615cbb162b0738f6e1fd1f5c3eda7e5da17861eacf4c717ea7"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:75723c3c0fbbf34350b46a3199eb50638ab22a0228f93fb472ef4d9becc2382b"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:83791a65b51ad6ee6cf0845634859d69a038ea9b03d7b26e703f94c7e93dbcf9"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0ef4854e82c09e84cc63084a9e4ccd6d9b154f1dbdd283efb92ecd0b5e2b8c84"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ed1184ab8f113e8d660ce49a56390ca181f2981066acc27cf637d5c1e10ce46e"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:d2997c458c690ec2bc6b0b7ecbafd02b029b7b4283078d3b32a852a7ce3ddd98"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:b58b4710c7f4161b5e9dcbe73bb7c62d65670a87df7bcce9e1faaad43e715245"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:0c009475ee389757e6e34611d75f6e4f05f0cf5ebb76c6037508318e1a1e0d7e"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:8dbf6d1bc73f1d04ec1734bae3b4fb0ee3cb2a493d35ede9badbeb901fb40f6f"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-win32.whl", hash = "sha256:3f78fd71c4f43a13d342be74ebbc0666fe1f555b8837eb113cb7416856c79682"}, + {file = "psycopg2_binary-2.9.9-cp310-cp310-win_amd64.whl", hash = "sha256:876801744b0dee379e4e3c38b76fc89f88834bb15bf92ee07d94acd06ec890a0"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:ee825e70b1a209475622f7f7b776785bd68f34af6e7a46e2e42f27b659b5bc26"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:1ea665f8ce695bcc37a90ee52de7a7980be5161375d42a0b6c6abedbf0d81f0f"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:143072318f793f53819048fdfe30c321890af0c3ec7cb1dfc9cc87aa88241de2"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c332c8d69fb64979ebf76613c66b985414927a40f8defa16cf1bc028b7b0a7b0"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f7fc5a5acafb7d6ccca13bfa8c90f8c51f13d8fb87d95656d3950f0158d3ce53"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:977646e05232579d2e7b9c59e21dbe5261f403a88417f6a6512e70d3f8a046be"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:b6356793b84728d9d50ead16ab43c187673831e9d4019013f1402c41b1db9b27"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:bc7bb56d04601d443f24094e9e31ae6deec9ccb23581f75343feebaf30423359"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:77853062a2c45be16fd6b8d6de2a99278ee1d985a7bd8b103e97e41c034006d2"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:78151aa3ec21dccd5cdef6c74c3e73386dcdfaf19bced944169697d7ac7482fc"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-win32.whl", hash = "sha256:dc4926288b2a3e9fd7b50dc6a1909a13bbdadfc67d93f3374d984e56f885579d"}, + {file = "psycopg2_binary-2.9.9-cp311-cp311-win_amd64.whl", hash = "sha256:b76bedd166805480ab069612119ea636f5ab8f8771e640ae103e05a4aae3e417"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:8532fd6e6e2dc57bcb3bc90b079c60de896d2128c5d9d6f24a63875a95a088cf"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:b0605eaed3eb239e87df0d5e3c6489daae3f7388d455d0c0b4df899519c6a38d"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8f8544b092a29a6ddd72f3556a9fcf249ec412e10ad28be6a0c0d948924f2212"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2d423c8d8a3c82d08fe8af900ad5b613ce3632a1249fd6a223941d0735fce493"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2e5afae772c00980525f6d6ecf7cbca55676296b580c0e6abb407f15f3706996"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6e6f98446430fdf41bd36d4faa6cb409f5140c1c2cf58ce0bbdaf16af7d3f119"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:c77e3d1862452565875eb31bdb45ac62502feabbd53429fdc39a1cc341d681ba"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:cb16c65dcb648d0a43a2521f2f0a2300f40639f6f8c1ecbc662141e4e3e1ee07"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-musllinux_1_1_ppc64le.whl", hash = "sha256:911dda9c487075abd54e644ccdf5e5c16773470a6a5d3826fda76699410066fb"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:57fede879f08d23c85140a360c6a77709113efd1c993923c59fde17aa27599fe"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-win32.whl", hash = "sha256:64cf30263844fa208851ebb13b0732ce674d8ec6a0c86a4e160495d299ba3c93"}, + {file = "psycopg2_binary-2.9.9-cp312-cp312-win_amd64.whl", hash = "sha256:81ff62668af011f9a48787564ab7eded4e9fb17a4a6a74af5ffa6a457400d2ab"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:2293b001e319ab0d869d660a704942c9e2cce19745262a8aba2115ef41a0a42a"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:03ef7df18daf2c4c07e2695e8cfd5ee7f748a1d54d802330985a78d2a5a6dca9"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0a602ea5aff39bb9fac6308e9c9d82b9a35c2bf288e184a816002c9fae930b77"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8359bf4791968c5a78c56103702000105501adb557f3cf772b2c207284273984"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:275ff571376626195ab95a746e6a04c7df8ea34638b99fc11160de91f2fef503"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:f9b5571d33660d5009a8b3c25dc1db560206e2d2f89d3df1cb32d72c0d117d52"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:420f9bbf47a02616e8554e825208cb947969451978dceb77f95ad09c37791dae"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:4154ad09dac630a0f13f37b583eae260c6aa885d67dfbccb5b02c33f31a6d420"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:a148c5d507bb9b4f2030a2025c545fccb0e1ef317393eaba42e7eabd28eb6041"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-win32.whl", hash = "sha256:68fc1f1ba168724771e38bee37d940d2865cb0f562380a1fb1ffb428b75cb692"}, + {file = "psycopg2_binary-2.9.9-cp37-cp37m-win_amd64.whl", hash = "sha256:281309265596e388ef483250db3640e5f414168c5a67e9c665cafce9492eda2f"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:60989127da422b74a04345096c10d416c2b41bd7bf2a380eb541059e4e999980"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:246b123cc54bb5361588acc54218c8c9fb73068bf227a4a531d8ed56fa3ca7d6"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:34eccd14566f8fe14b2b95bb13b11572f7c7d5c36da61caf414d23b91fcc5d94"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:18d0ef97766055fec15b5de2c06dd8e7654705ce3e5e5eed3b6651a1d2a9a152"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d3f82c171b4ccd83bbaf35aa05e44e690113bd4f3b7b6cc54d2219b132f3ae55"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ead20f7913a9c1e894aebe47cccf9dc834e1618b7aa96155d2091a626e59c972"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:ca49a8119c6cbd77375ae303b0cfd8c11f011abbbd64601167ecca18a87e7cdd"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:323ba25b92454adb36fa425dc5cf6f8f19f78948cbad2e7bc6cdf7b0d7982e59"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:1236ed0952fbd919c100bc839eaa4a39ebc397ed1c08a97fc45fee2a595aa1b3"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:729177eaf0aefca0994ce4cffe96ad3c75e377c7b6f4efa59ebf003b6d398716"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-win32.whl", hash = "sha256:804d99b24ad523a1fe18cc707bf741670332f7c7412e9d49cb5eab67e886b9b5"}, + {file = "psycopg2_binary-2.9.9-cp38-cp38-win_amd64.whl", hash = "sha256:a6cdcc3ede532f4a4b96000b6362099591ab4a3e913d70bcbac2b56c872446f7"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:72dffbd8b4194858d0941062a9766f8297e8868e1dd07a7b36212aaa90f49472"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:30dcc86377618a4c8f3b72418df92e77be4254d8f89f14b8e8f57d6d43603c0f"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:31a34c508c003a4347d389a9e6fcc2307cc2150eb516462a7a17512130de109e"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:15208be1c50b99203fe88d15695f22a5bed95ab3f84354c494bcb1d08557df67"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1873aade94b74715be2246321c8650cabf5a0d098a95bab81145ffffa4c13876"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3a58c98a7e9c021f357348867f537017057c2ed7f77337fd914d0bedb35dace7"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:4686818798f9194d03c9129a4d9a702d9e113a89cb03bffe08c6cf799e053291"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:ebdc36bea43063116f0486869652cb2ed7032dbc59fbcb4445c4862b5c1ecf7f"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:ca08decd2697fdea0aea364b370b1249d47336aec935f87b8bbfd7da5b2ee9c1"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:ac05fb791acf5e1a3e39402641827780fe44d27e72567a000412c648a85ba860"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-win32.whl", hash = "sha256:9dba73be7305b399924709b91682299794887cbbd88e38226ed9f6712eabee90"}, + {file = "psycopg2_binary-2.9.9-cp39-cp39-win_amd64.whl", hash = "sha256:f7ae5d65ccfbebdfa761585228eb4d0df3a8b15cfb53bd953e713e09fbb12957"}, +] + [[package]] name = "py4j" version = "0.10.9.7" @@ -225,36 +303,47 @@ files = [ [[package]] name = "pyarrow" -version = "12.0.1" +version = "14.0.1" description = "Python library for Apache Arrow" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "pyarrow-12.0.1-cp310-cp310-macosx_10_14_x86_64.whl", hash = "sha256:6d288029a94a9bb5407ceebdd7110ba398a00412c5b0155ee9813a40d246c5df"}, - {file = "pyarrow-12.0.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:345e1828efdbd9aa4d4de7d5676778aba384a2c3add896d995b23d368e60e5af"}, - {file = "pyarrow-12.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8d6009fdf8986332b2169314da482baed47ac053311c8934ac6651e614deacd6"}, - {file = "pyarrow-12.0.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2d3c4cbbf81e6dd23fe921bc91dc4619ea3b79bc58ef10bce0f49bdafb103daf"}, - {file = "pyarrow-12.0.1-cp310-cp310-win_amd64.whl", hash = "sha256:cdacf515ec276709ac8042c7d9bd5be83b4f5f39c6c037a17a60d7ebfd92c890"}, - {file = "pyarrow-12.0.1-cp311-cp311-macosx_10_14_x86_64.whl", hash = "sha256:749be7fd2ff260683f9cc739cb862fb11be376de965a2a8ccbf2693b098db6c7"}, - {file = "pyarrow-12.0.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:6895b5fb74289d055c43db3af0de6e16b07586c45763cb5e558d38b86a91e3a7"}, - {file = "pyarrow-12.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1887bdae17ec3b4c046fcf19951e71b6a619f39fa674f9881216173566c8f718"}, - {file = "pyarrow-12.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e2c9cb8eeabbadf5fcfc3d1ddea616c7ce893db2ce4dcef0ac13b099ad7ca082"}, - {file = "pyarrow-12.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:ce4aebdf412bd0eeb800d8e47db854f9f9f7e2f5a0220440acf219ddfddd4f63"}, - {file = "pyarrow-12.0.1-cp37-cp37m-macosx_10_14_x86_64.whl", hash = "sha256:e0d8730c7f6e893f6db5d5b86eda42c0a130842d101992b581e2138e4d5663d3"}, - {file = "pyarrow-12.0.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:43364daec02f69fec89d2315f7fbfbeec956e0d991cbbef471681bd77875c40f"}, - {file = "pyarrow-12.0.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:051f9f5ccf585f12d7de836e50965b3c235542cc896959320d9776ab93f3b33d"}, - {file = "pyarrow-12.0.1-cp37-cp37m-win_amd64.whl", hash = "sha256:be2757e9275875d2a9c6e6052ac7957fbbfc7bc7370e4a036a9b893e96fedaba"}, - {file = "pyarrow-12.0.1-cp38-cp38-macosx_10_14_x86_64.whl", hash = "sha256:cf812306d66f40f69e684300f7af5111c11f6e0d89d6b733e05a3de44961529d"}, - {file = "pyarrow-12.0.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:459a1c0ed2d68671188b2118c63bac91eaef6fc150c77ddd8a583e3c795737bf"}, - {file = "pyarrow-12.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:85e705e33eaf666bbe508a16fd5ba27ca061e177916b7a317ba5a51bee43384c"}, - {file = "pyarrow-12.0.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9120c3eb2b1f6f516a3b7a9714ed860882d9ef98c4b17edcdc91d95b7528db60"}, - {file = "pyarrow-12.0.1-cp38-cp38-win_amd64.whl", hash = "sha256:c780f4dc40460015d80fcd6a6140de80b615349ed68ef9adb653fe351778c9b3"}, - {file = "pyarrow-12.0.1-cp39-cp39-macosx_10_14_x86_64.whl", hash = "sha256:a3c63124fc26bf5f95f508f5d04e1ece8cc23a8b0af2a1e6ab2b1ec3fdc91b24"}, - {file = "pyarrow-12.0.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:b13329f79fa4472324f8d32dc1b1216616d09bd1e77cfb13104dec5463632c36"}, - {file = "pyarrow-12.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bb656150d3d12ec1396f6dde542db1675a95c0cc8366d507347b0beed96e87ca"}, - {file = "pyarrow-12.0.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6251e38470da97a5b2e00de5c6a049149f7b2bd62f12fa5dbb9ac674119ba71a"}, - {file = "pyarrow-12.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:3de26da901216149ce086920547dfff5cd22818c9eab67ebc41e863a5883bac7"}, - {file = "pyarrow-12.0.1.tar.gz", hash = "sha256:cce317fc96e5b71107bf1f9f184d5e54e2bd14bbf3f9a3d62819961f0af86fec"}, + {file = "pyarrow-14.0.1-cp310-cp310-macosx_10_14_x86_64.whl", hash = "sha256:96d64e5ba7dceb519a955e5eeb5c9adcfd63f73a56aea4722e2cc81364fc567a"}, + {file = "pyarrow-14.0.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1a8ae88c0038d1bc362a682320112ee6774f006134cd5afc291591ee4bc06505"}, + {file = "pyarrow-14.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0f6f053cb66dc24091f5511e5920e45c83107f954a21032feadc7b9e3a8e7851"}, + {file = "pyarrow-14.0.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:906b0dc25f2be12e95975722f1e60e162437023f490dbd80d0deb7375baf3171"}, + {file = "pyarrow-14.0.1-cp310-cp310-manylinux_2_28_aarch64.whl", hash = "sha256:78d4a77a46a7de9388b653af1c4ce539350726cd9af62e0831e4f2bd0c95a2f4"}, + {file = "pyarrow-14.0.1-cp310-cp310-manylinux_2_28_x86_64.whl", hash = "sha256:06ca79080ef89d6529bb8e5074d4b4f6086143b2520494fcb7cf8a99079cde93"}, + {file = "pyarrow-14.0.1-cp310-cp310-win_amd64.whl", hash = "sha256:32542164d905002c42dff896efdac79b3bdd7291b1b74aa292fac8450d0e4dcd"}, + {file = "pyarrow-14.0.1-cp311-cp311-macosx_10_14_x86_64.whl", hash = "sha256:c7331b4ed3401b7ee56f22c980608cf273f0380f77d0f73dd3c185f78f5a6220"}, + {file = "pyarrow-14.0.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:922e8b49b88da8633d6cac0e1b5a690311b6758d6f5d7c2be71acb0f1e14cd61"}, + {file = "pyarrow-14.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:58c889851ca33f992ea916b48b8540735055201b177cb0dcf0596a495a667b00"}, + {file = "pyarrow-14.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:30d8494870d9916bb53b2a4384948491444741cb9a38253c590e21f836b01222"}, + {file = "pyarrow-14.0.1-cp311-cp311-manylinux_2_28_aarch64.whl", hash = "sha256:be28e1a07f20391bb0b15ea03dcac3aade29fc773c5eb4bee2838e9b2cdde0cb"}, + {file = "pyarrow-14.0.1-cp311-cp311-manylinux_2_28_x86_64.whl", hash = "sha256:981670b4ce0110d8dcb3246410a4aabf5714db5d8ea63b15686bce1c914b1f83"}, + {file = "pyarrow-14.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:4756a2b373a28f6166c42711240643fb8bd6322467e9aacabd26b488fa41ec23"}, + {file = "pyarrow-14.0.1-cp312-cp312-macosx_10_14_x86_64.whl", hash = "sha256:cf87e2cec65dd5cf1aa4aba918d523ef56ef95597b545bbaad01e6433851aa10"}, + {file = "pyarrow-14.0.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:470ae0194fbfdfbf4a6b65b4f9e0f6e1fa0ea5b90c1ee6b65b38aecee53508c8"}, + {file = "pyarrow-14.0.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6263cffd0c3721c1e348062997babdf0151301f7353010c9c9a8ed47448f82ab"}, + {file = "pyarrow-14.0.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7a8089d7e77d1455d529dbd7cff08898bbb2666ee48bc4085203af1d826a33cc"}, + {file = "pyarrow-14.0.1-cp312-cp312-manylinux_2_28_aarch64.whl", hash = "sha256:fada8396bc739d958d0b81d291cfd201126ed5e7913cb73de6bc606befc30226"}, + {file = "pyarrow-14.0.1-cp312-cp312-manylinux_2_28_x86_64.whl", hash = "sha256:2a145dab9ed7849fc1101bf03bcdc69913547f10513fdf70fc3ab6c0a50c7eee"}, + {file = "pyarrow-14.0.1-cp312-cp312-win_amd64.whl", hash = "sha256:05fe7994745b634c5fb16ce5717e39a1ac1fac3e2b0795232841660aa76647cd"}, + {file = "pyarrow-14.0.1-cp38-cp38-macosx_10_14_x86_64.whl", hash = "sha256:a8eeef015ae69d104c4c3117a6011e7e3ecd1abec79dc87fd2fac6e442f666ee"}, + {file = "pyarrow-14.0.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:3c76807540989fe8fcd02285dd15e4f2a3da0b09d27781abec3adc265ddbeba1"}, + {file = "pyarrow-14.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:450e4605e3c20e558485f9161a79280a61c55efe585d51513c014de9ae8d393f"}, + {file = "pyarrow-14.0.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:323cbe60210173ffd7db78bfd50b80bdd792c4c9daca8843ef3cd70b186649db"}, + {file = "pyarrow-14.0.1-cp38-cp38-manylinux_2_28_aarch64.whl", hash = "sha256:0140c7e2b740e08c5a459439d87acd26b747fc408bde0a8806096ee0baaa0c15"}, + {file = "pyarrow-14.0.1-cp38-cp38-manylinux_2_28_x86_64.whl", hash = "sha256:e592e482edd9f1ab32f18cd6a716c45b2c0f2403dc2af782f4e9674952e6dd27"}, + {file = "pyarrow-14.0.1-cp38-cp38-win_amd64.whl", hash = "sha256:d264ad13605b61959f2ae7c1d25b1a5b8505b112715c961418c8396433f213ad"}, + {file = "pyarrow-14.0.1-cp39-cp39-macosx_10_14_x86_64.whl", hash = "sha256:01e44de9749cddc486169cb632f3c99962318e9dacac7778315a110f4bf8a450"}, + {file = "pyarrow-14.0.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:d0351fecf0e26e152542bc164c22ea2a8e8c682726fce160ce4d459ea802d69c"}, + {file = "pyarrow-14.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:33c1f6110c386464fd2e5e4ea3624466055bbe681ff185fd6c9daa98f30a3f9a"}, + {file = "pyarrow-14.0.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:11e045dfa09855b6d3e7705a37c42e2dc2c71d608fab34d3c23df2e02df9aec3"}, + {file = "pyarrow-14.0.1-cp39-cp39-manylinux_2_28_aarch64.whl", hash = "sha256:097828b55321897db0e1dbfc606e3ff8101ae5725673498cbfa7754ee0da80e4"}, + {file = "pyarrow-14.0.1-cp39-cp39-manylinux_2_28_x86_64.whl", hash = "sha256:1daab52050a1c48506c029e6fa0944a7b2436334d7e44221c16f6f1b2cc9c510"}, + {file = "pyarrow-14.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:3f6d5faf4f1b0d5a7f97be987cf9e9f8cd39902611e818fe134588ee99bf0283"}, + {file = "pyarrow-14.0.1.tar.gz", hash = "sha256:b8b3f4fe8d4ec15e1ef9b599b94683c5216adaed78d5cb4c606180546d1e2ee1"}, ] [package.dependencies] @@ -336,4 +425,4 @@ files = [ [metadata] lock-version = "2.0" python-versions = "^3.11" -content-hash = "419289b35beab928ed8e691e696517bd8b1c093630d90d8d8568c6b9ba1b9f2b" +content-hash = "42b788fcc212aceceeff3c9e2743d39c048a323068a3b49940b7efa830e6863e" From 950ed58b5fcab88ca00a08c7683120a2d042b54d Mon Sep 17 00:00:00 2001 From: Phoenix7Rise <81153420+keven-huang@users.noreply.github.com> Date: Mon, 13 Nov 2023 17:49:08 +0800 Subject: [PATCH 48/77] fix(conn): fix throw error msg of connector when encode Json or Csv (#13390) Co-authored-by: jiamin.huang --- src/connector/src/parser/plain_parser.rs | 2 +- .../tests/testdata/output/create_source.yaml | 16 +++++++-------- .../testdata/output/emit_on_window_close.yaml | 20 +++++++++---------- .../tests/testdata/output/explain.yaml | 8 ++++---- .../testdata/output/generated_columns.yaml | 2 +- .../tests/testdata/output/watermark.yaml | 18 ++++++++--------- src/frontend/src/handler/create_source.rs | 4 +++- 7 files changed, 36 insertions(+), 34 deletions(-) diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 7448efb6f19d2..cd805af4a1893 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -70,7 +70,7 @@ impl PlainParser { payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, ) -> Result<()> { - // if key is empty, set it as vec![]su + // if key is empty, set it as vec![] let key_data = key.unwrap_or_default(); // if payload is empty, report error let payload_data = payload.ok_or_else(|| { diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index c20af237eadc1..fef47aa736aff 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -20,11 +20,11 @@ scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE CSV (delimiter = ',', without_header = true); explain_output: | - StreamMaterialize { columns: [v1, v2, _rw_key(hidden), _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } + StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamDml { columns: [v1, v2, _rw_key, _row_id] } - └─StreamSource { source: s0, columns: [v1, v2, _rw_key, _row_id] } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource { source: s0, columns: [v1, v2, _row_id] } - id: csv_delimiter_tab sql: | explain create table s0 (v1 int, v2 varchar) with ( @@ -34,11 +34,11 @@ scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE CSV (delimiter = E'\t', without_header = true); explain_output: | - StreamMaterialize { columns: [v1, v2, _rw_key(hidden), _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } + StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamDml { columns: [v1, v2, _rw_key, _row_id] } - └─StreamSource { source: s0, columns: [v1, v2, _rw_key, _row_id] } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource { source: s0, columns: [v1, v2, _row_id] } - id: create_source_with_cdc_backfill sql: | create source mysql_mydb with ( diff --git a/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml b/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml index cb7155cfa0a4f..cd3019382bd66 100644 --- a/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml +++ b/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml @@ -19,17 +19,17 @@ └─StreamProject { exprs: [v1, min(v2), count(distinct v3)], output_watermarks: [v1] } └─StreamHashAgg [append_only] { group_key: [v1], aggs: [min(v2), count(distinct v3), count], output_watermarks: [v1] } └─StreamExchange { dist: HashShard(v1) } - └─StreamRowIdGen { row_id_index: 4 } + └─StreamRowIdGen { row_id_index: 3 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - 10:Int32) }], output_watermarks: [v1] } - └─StreamSource { source: t, columns: [v1, v2, v3, _rw_key, _row_id] } + └─StreamSource { source: t, columns: [v1, v2, v3, _row_id] } eowc_stream_plan: |- StreamMaterialize { columns: [v1, min, agg], stream_key: [v1], pk_columns: [v1], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamProject { exprs: [v1, min(v2), count(distinct v3)], output_watermarks: [v1] } └─StreamHashAgg [append_only, eowc] { group_key: [v1], aggs: [min(v2), count(distinct v3), count], output_watermarks: [v1] } └─StreamExchange { dist: HashShard(v1) } - └─StreamRowIdGen { row_id_index: 4 } + └─StreamRowIdGen { row_id_index: 3 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - 10:Int32) }], output_watermarks: [v1] } - └─StreamSource { source: t, columns: [v1, v2, v3, _rw_key, _row_id] } + └─StreamSource { source: t, columns: [v1, v2, v3, _row_id] } eowc_stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [v1, min, agg], stream_key: [v1], pk_columns: [v1], pk_conflict: NoCheck, watermark_columns: [v1] } @@ -42,10 +42,10 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamRowIdGen { row_id_index: 4 } + StreamRowIdGen { row_id_index: 3 } └── StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - 10:Int32) }], output_watermarks: [v1] } ├── state tables: [ 2 ] - └── StreamSource { source: t, columns: [v1, v2, v3, _rw_key, _row_id] } { source state table: 3 } + └── StreamSource { source: t, columns: [v1, v2, v3, _row_id] } { source state table: 3 } Table 0 ├── columns: [ v1, min(v2), count(distinct v3), count ] @@ -142,9 +142,9 @@ └─StreamEowcSort { sort_column: tm } └─StreamExchange { dist: HashShard(b) } └─StreamProject { exprs: [a, b, tm, _row_id], output_watermarks: [tm] } - └─StreamRowIdGen { row_id_index: 4 } + └─StreamRowIdGen { row_id_index: 3 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: tm, expr: (tm - '00:05:00':Interval) }], output_watermarks: [tm] } - └─StreamSource { source: t, columns: [a, b, tm, _rw_key, _row_id] } + └─StreamSource { source: t, columns: [a, b, tm, _row_id] } eowc_stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [lag, _row_id(hidden), b(hidden)], stream_key: [_row_id, b], pk_columns: [_row_id, b], pk_conflict: NoCheck } @@ -157,10 +157,10 @@ Fragment 1 StreamProject { exprs: [a, b, tm, _row_id], output_watermarks: [tm] } - └── StreamRowIdGen { row_id_index: 4 } + └── StreamRowIdGen { row_id_index: 3 } └── StreamWatermarkFilter { watermark_descs: [Desc { column: tm, expr: (tm - '00:05:00':Interval) }], output_watermarks: [tm] } ├── state tables: [ 2 ] - └── StreamSource { source: t, columns: [a, b, tm, _rw_key, _row_id] } { source state table: 3 } + └── StreamSource { source: t, columns: [a, b, tm, _row_id] } { source state table: 3 } Table 0 ├── columns: [ a, b, tm, _row_id ] diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index d3ac80df9b574..63899bc26c0e4 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -198,8 +198,8 @@ - sql: | explain create table t (v1 int, v2 varchar) with ( connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest' ) FORMAT PLAIN ENCODE JSON; explain_output: | - StreamMaterialize { columns: [v1, v2, _rw_key(hidden), _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } + StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamDml { columns: [v1, v2, _rw_key, _row_id] } - └─StreamSource { source: t, columns: [v1, v2, _rw_key, _row_id] } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource { source: t, columns: [v1, v2, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml index bb91f726670a9..a3acaf174d237 100644 --- a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml +++ b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml @@ -16,7 +16,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [(v2 + 1:Int32) as $expr1] } - └─BatchSource { source: s1, columns: [v2, _rw_key, _row_id], filter: (None, None) } + └─BatchSource { source: s1, columns: [v2, _row_id], filter: (None, None) } - name: select proctime() sql: | select proctime(); diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml index e56f4eff7496f..d57d41fa76bc3 100644 --- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml @@ -5,22 +5,22 @@ select t.v1 - INTERVAL '2' SECOND as v1 from t; logical_plan: |- LogicalProject { exprs: [(v1 - '00:00:02':Interval) as $expr1] } - └─LogicalSource { source: t, columns: [v1, _rw_key, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: t, columns: [v1, _row_id], time_range: (Unbounded, Unbounded) } stream_plan: |- StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamProject { exprs: [SubtractWithTimeZone(v1, '00:00:02':Interval, 'UTC':Varchar) as $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamRowIdGen { row_id_index: 2 } + └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - '00:00:01':Interval) }], output_watermarks: [v1] } - └─StreamSource { source: t, columns: [v1, _rw_key, _row_id] } + └─StreamSource { source: t, columns: [v1, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } ├── materialized table: 4294967294 └── StreamProject { exprs: [SubtractWithTimeZone(v1, '00:00:02':Interval, 'UTC':Varchar) as $expr1, _row_id], output_watermarks: [$expr1] } - └── StreamRowIdGen { row_id_index: 2 } + └── StreamRowIdGen { row_id_index: 1 } └── StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - '00:00:01':Interval) }], output_watermarks: [v1] } ├── state tables: [ 0 ] - └── StreamSource { source: t, columns: [v1, _rw_key, _row_id] } { source state table: 1 } + └── StreamSource { source: t, columns: [v1, _row_id] } { source state table: 1 } Table 0 ├── columns: [ vnode, offset ] @@ -48,11 +48,11 @@ sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only with (connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest') FORMAT PLAIN ENCODE JSON; explain_output: | - StreamMaterialize { columns: [v1, _rw_key(hidden), _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } - └─StreamRowIdGen { row_id_index: 2 } + StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } + └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - '00:00:01':Interval) }], output_watermarks: [v1] } - └─StreamDml { columns: [v1, _rw_key, _row_id] } - └─StreamSource { source: t, columns: [v1, _rw_key, _row_id] } + └─StreamDml { columns: [v1, _row_id] } + └─StreamSource { source: t, columns: [v1, _row_id] } - name: watermark on append only table without source sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index d9ff525b5d85e..e93d703671446 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -718,7 +718,9 @@ pub(crate) async fn bind_source_pk( let sql_defined_pk = !sql_defined_pk_names.is_empty(); let res = match (&source_schema.format, &source_schema.row_encode) { - (Format::Native, Encode::Native) => sql_defined_pk_names, + (Format::Native, Encode::Native) | (Format::Plain, Encode::Json | Encode::Csv) => { + sql_defined_pk_names + } (Format::Plain, _) => { if is_key_mq_connector(with_properties) { add_default_key_column(columns); From d932c58853d3fc0f7d371d2152eb12146446ba47 Mon Sep 17 00:00:00 2001 From: lmatz Date: Mon, 13 Nov 2023 20:35:45 +0800 Subject: [PATCH 49/77] chore: update image version in docker (#13357) --- docker/docker-compose-standalone.yml | 2 +- docker/docker-compose.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/docker-compose-standalone.yml b/docker/docker-compose-standalone.yml index b870589713d11..f7dcb9b576ebf 100644 --- a/docker/docker-compose-standalone.yml +++ b/docker/docker-compose-standalone.yml @@ -1,7 +1,7 @@ --- version: "3" x-image: &image - image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.3.0} + image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.4.0} services: risingwave-standalone: <<: *image diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index fd1532b0728ad..36c67d7235b61 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -1,7 +1,7 @@ --- version: "3" x-image: &image - image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.3.0} + image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.4.0} services: compactor-0: <<: *image From aaccc89203fcf711750f2aacf9b08a5bf7124e50 Mon Sep 17 00:00:00 2001 From: Croxx Date: Mon, 13 Nov 2023 21:20:36 +0800 Subject: [PATCH 50/77] refactor(storage): avoid decode and compression when refill (#13309) Signed-off-by: MrCroxx --- Cargo.lock | 15 +-- src/storage/Cargo.toml | 2 +- .../src/hummock/event_handler/refiller.rs | 22 +-- src/storage/src/hummock/file_cache/store.rs | 127 ++++++++++++++++-- src/storage/src/hummock/sstable_store.rs | 20 +-- 5 files changed, 147 insertions(+), 39 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index d662200755408..b2928f34902cb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3125,7 +3125,7 @@ dependencies = [ [[package]] name = "foyer" version = "0.1.0" -source = "git+https://github.com/MrCroxx/foyer?rev=9232b3a#9232b3aa468460b60c44c26eac62dadf3594a383" +source = "git+https://github.com/MrCroxx/foyer?rev=ce2e222#ce2e22292e39746a6792e74c09718855d06dd0a4" dependencies = [ "foyer-common", "foyer-intrusive", @@ -3136,8 +3136,9 @@ dependencies = [ [[package]] name = "foyer-common" version = "0.1.0" -source = "git+https://github.com/MrCroxx/foyer?rev=9232b3a#9232b3aa468460b60c44c26eac62dadf3594a383" +source = "git+https://github.com/MrCroxx/foyer?rev=ce2e222#ce2e22292e39746a6792e74c09718855d06dd0a4" dependencies = [ + "anyhow", "bytes", "foyer-workspace-hack", "itertools 0.11.0", @@ -3151,7 +3152,7 @@ dependencies = [ [[package]] name = "foyer-intrusive" version = "0.1.0" -source = "git+https://github.com/MrCroxx/foyer?rev=9232b3a#9232b3aa468460b60c44c26eac62dadf3594a383" +source = "git+https://github.com/MrCroxx/foyer?rev=ce2e222#ce2e22292e39746a6792e74c09718855d06dd0a4" dependencies = [ "bytes", "cmsketch", @@ -3168,7 +3169,7 @@ dependencies = [ [[package]] name = "foyer-storage" version = "0.1.0" -source = "git+https://github.com/MrCroxx/foyer?rev=9232b3a#9232b3aa468460b60c44c26eac62dadf3594a383" +source = "git+https://github.com/MrCroxx/foyer?rev=ce2e222#ce2e22292e39746a6792e74c09718855d06dd0a4" dependencies = [ "anyhow", "bitflags 2.4.0", @@ -3181,6 +3182,7 @@ dependencies = [ "futures", "itertools 0.11.0", "libc", + "lz4", "madsim-tokio", "memoffset", "nix 0.27.1", @@ -3197,7 +3199,7 @@ dependencies = [ [[package]] name = "foyer-workspace-hack" version = "0.1.0" -source = "git+https://github.com/MrCroxx/foyer?rev=9232b3a#9232b3aa468460b60c44c26eac62dadf3594a383" +source = "git+https://github.com/MrCroxx/foyer?rev=ce2e222#ce2e22292e39746a6792e74c09718855d06dd0a4" dependencies = [ "cc", "crossbeam-utils", @@ -3216,9 +3218,6 @@ dependencies = [ "proc-macro2", "quote", "rand", - "regex", - "regex-automata 0.4.1", - "regex-syntax 0.8.0", "syn 2.0.37", "tokio", "tracing", diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index a3899824e9c87..6150a6e862fdc 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -26,7 +26,7 @@ dyn-clone = "1.0.14" either = "1" enum-as-inner = "0.6" fail = "0.5" -foyer = { git = "https://github.com/MrCroxx/foyer", rev = "9232b3a" } +foyer = { git = "https://github.com/MrCroxx/foyer", rev = "ce2e222" } futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } hex = "0.4" diff --git a/src/storage/src/hummock/event_handler/refiller.rs b/src/storage/src/hummock/event_handler/refiller.rs index e9eabb952d2d2..5a11153371db3 100644 --- a/src/storage/src/hummock/event_handler/refiller.rs +++ b/src/storage/src/hummock/event_handler/refiller.rs @@ -20,6 +20,7 @@ use std::task::{ready, Context, Poll}; use std::time::{Duration, Instant}; use foyer::common::code::Key; +use foyer::common::range::RangeBoundsExt; use futures::future::{join_all, try_join_all}; use futures::{Future, FutureExt}; use itertools::Itertools; @@ -37,7 +38,8 @@ use tokio::task::JoinHandle; use crate::hummock::file_cache::preclude::*; use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::{ - Block, HummockError, HummockResult, Sstable, SstableBlockIndex, SstableStoreRef, TableHolder, + CachedBlock, FileCacheCompression, HummockError, HummockResult, Sstable, SstableBlockIndex, + SstableStoreRef, TableHolder, }; use crate::monitor::StoreLocalStatistic; @@ -382,14 +384,15 @@ impl CacheRefillTask { let mut admits = 0; for block_index in block_index_start..block_index_end { - let (range, uncompressed_capacity) = sst.calculate_block_info(block_index); + let (range, _uncompressed_capacity) = sst.calculate_block_info(block_index); let key = SstableBlockIndex { sst_id: object_id, block_idx: block_index as u64, }; + // see `CachedBlock::serialized_len()` let mut writer = sstable_store .data_file_cache() - .writer(key, key.serialized_len() + uncompressed_capacity); + .writer(key, key.serialized_len() + 1 + 8 + range.size().unwrap()); if writer.judge() { admits += 1; @@ -422,13 +425,16 @@ impl CacheRefillTask { let bytes = data.slice(offset..offset + len); let future = async move { - let block = Block::decode( + let value = CachedBlock::Fetched { bytes, - writer.weight() - writer.key().serialized_len(), - )?; - let block = Box::new(block); + uncompressed_capacity: writer.weight() + - writer.key().serialized_len(), + }; writer.force(); - let res = writer.finish(block).await.map_err(HummockError::file_cache); + // TODO(MrCroxx): compress if raw is not compressed? + // skip compression for it may already be compressed. + writer.set_compression(FileCacheCompression::None); + let res = writer.finish(value).await.map_err(HummockError::file_cache); if matches!(res, Ok(true)) { GLOBAL_CACHE_REFILL_METRICS .data_refill_success_bytes diff --git a/src/storage/src/hummock/file_cache/store.rs b/src/storage/src/hummock/file_cache/store.rs index 59a84d2d80487..c3f8edeae933e 100644 --- a/src/storage/src/hummock/file_cache/store.rs +++ b/src/storage/src/hummock/file_cache/store.rs @@ -18,7 +18,7 @@ use std::path::PathBuf; use std::sync::Arc; use bytes::{Buf, BufMut, Bytes}; -use foyer::common::code::{Key, Value}; +use foyer::common::code::{CodingResult, Key, Value}; use foyer::intrusive::eviction::lfu::LfuConfig; use foyer::storage::admission::rated_ticket::RatedTicketAdmissionPolicy; use foyer::storage::admission::AdmissionPolicy; @@ -48,6 +48,7 @@ pub type DeviceConfig = foyer::storage::device::fs::FsDeviceConfig; pub type FileCacheResult = foyer::storage::error::Result; pub type FileCacheError = foyer::storage::error::Error; +pub type FileCacheCompression = foyer::storage::compress::Compression; #[derive(Debug)] pub struct FileCacheConfig @@ -180,6 +181,20 @@ where FileCacheWriter::None { writer } => writer.finish(value).await, } } + + fn compression(&self) -> Compression { + match self { + FileCacheWriter::Foyer { writer } => writer.compression(), + FileCacheWriter::None { writer } => writer.compression(), + } + } + + fn set_compression(&mut self, compression: Compression) { + match self { + FileCacheWriter::Foyer { writer } => writer.set_compression(compression), + FileCacheWriter::None { writer } => writer.set_compression(compression), + } + } } #[derive(Debug)] @@ -336,15 +351,95 @@ impl Key for SstableBlockIndex { 8 + 8 // sst_id (8B) + block_idx (8B) } - fn write(&self, mut buf: &mut [u8]) { + fn write(&self, mut buf: &mut [u8]) -> CodingResult<()> { buf.put_u64(self.sst_id); buf.put_u64(self.block_idx); + Ok(()) } - fn read(mut buf: &[u8]) -> Self { + fn read(mut buf: &[u8]) -> CodingResult { let sst_id = buf.get_u64(); let block_idx = buf.get_u64(); - Self { sst_id, block_idx } + Ok(Self { sst_id, block_idx }) + } +} + +/// [`CachedBlock`] uses different coding for writing to use/bypass compression. +/// +/// But when reading, it will always be `Loaded`. +#[derive(Debug)] +pub enum CachedBlock { + Loaded { + block: Box, + }, + Fetched { + bytes: Bytes, + uncompressed_capacity: usize, + }, +} + +impl CachedBlock { + pub fn should_compress(&self) -> bool { + match self { + CachedBlock::Loaded { .. } => true, + // TODO(MrCroxx): based on block original compression algorithm? + CachedBlock::Fetched { .. } => false, + } + } + + pub fn into_inner(self) -> Box { + match self { + CachedBlock::Loaded { block } => block, + CachedBlock::Fetched { .. } => unreachable!(), + } + } +} + +impl Value for CachedBlock { + fn serialized_len(&self) -> usize { + 1 /* type */ + match self { + CachedBlock::Loaded { block } => block.raw_data().len(), + CachedBlock::Fetched { bytes, uncompressed_capacity: _ } => 8 + bytes.len(), + } + } + + fn write(&self, mut buf: &mut [u8]) -> CodingResult<()> { + match self { + CachedBlock::Loaded { block } => { + buf.put_u8(0); + buf.put_slice(block.raw_data()) + } + CachedBlock::Fetched { + bytes, + uncompressed_capacity, + } => { + buf.put_u8(1); + buf.put_u64(*uncompressed_capacity as u64); + buf.put_slice(bytes); + } + } + Ok(()) + } + + fn read(mut buf: &[u8]) -> CodingResult { + let v = buf.get_u8(); + let res = match v { + 0 => { + let data = Bytes::copy_from_slice(buf); + let block = Block::decode_from_raw(data); + let block = Box::new(block); + Self::Loaded { block } + } + 1 => { + let uncompressed_capacity = buf.get_u64() as usize; + let data = Bytes::copy_from_slice(buf); + let block = Block::decode(data, uncompressed_capacity)?; + let block = Box::new(block); + Self::Loaded { block } + } + _ => unreachable!(), + }; + Ok(res) } } @@ -353,14 +448,16 @@ impl Value for Box { self.raw_data().len() } - fn write(&self, mut buf: &mut [u8]) { - buf.put_slice(self.raw_data()) + fn write(&self, mut buf: &mut [u8]) -> CodingResult<()> { + buf.put_slice(self.raw_data()); + Ok(()) } - fn read(buf: &[u8]) -> Self { + fn read(buf: &[u8]) -> CodingResult { let data = Bytes::copy_from_slice(buf); let block = Block::decode_from_raw(data); - Box::new(block) + let block = Box::new(block); + Ok(block) } } @@ -369,18 +466,20 @@ impl Value for Box { 8 + self.meta.encoded_size() // id (8B) + meta size } - fn write(&self, mut buf: &mut [u8]) { + fn write(&self, mut buf: &mut [u8]) -> CodingResult<()> { buf.put_u64(self.id); // TODO(MrCroxx): avoid buffer copy let mut buffer = vec![]; self.meta.encode_to(&mut buffer); - buf.put_slice(&buffer[..]) + buf.put_slice(&buffer[..]); + Ok(()) } - fn read(mut buf: &[u8]) -> Self { + fn read(mut buf: &[u8]) -> CodingResult { let id = buf.get_u64(); let meta = SstableMeta::decode(buf).unwrap(); - Box::new(Sstable::new(id, meta)) + let sstable = Box::new(Sstable::new(id, meta)); + Ok(sstable) } } @@ -416,9 +515,9 @@ mod tests { ); let mut buf = vec![0; block.serialized_len()]; - block.write(&mut buf[..]); + block.write(&mut buf[..]).unwrap(); - let block = as Value>::read(&buf[..]); + let block = as Value>::read(&buf[..]).unwrap(); let mut bi = BlockIterator::new(BlockHolder::from_owned_block(block)); diff --git a/src/storage/src/hummock/sstable_store.rs b/src/storage/src/hummock/sstable_store.rs index 25ac82636c77d..fe0e23e695845 100644 --- a/src/storage/src/hummock/sstable_store.rs +++ b/src/storage/src/hummock/sstable_store.rs @@ -36,7 +36,7 @@ use zstd::zstd_safe::WriteBuf; use super::utils::MemoryTracker; use super::{ - Block, BlockCache, BlockMeta, BlockResponse, FileCache, RecentFilter, Sstable, + Block, BlockCache, BlockMeta, BlockResponse, CachedBlock, FileCache, RecentFilter, Sstable, SstableBlockIndex, SstableMeta, SstableWriter, }; use crate::hummock::file_cache::preclude::*; @@ -94,7 +94,7 @@ impl From for TracedCachePolicy { } struct BlockCacheEventListener { - data_file_cache: FileCache>, + data_file_cache: FileCache, } impl LruCacheEventListener for BlockCacheEventListener { @@ -106,7 +106,8 @@ impl LruCacheEventListener for BlockCacheEventListener { sst_id: key.0, block_idx: key.1, }; - self.data_file_cache.insert_async(key, value); + self.data_file_cache + .insert_if_not_exists_async(key, CachedBlock::Loaded { block: value }); } } @@ -117,7 +118,7 @@ impl LruCacheEventListener for MetaCacheEventListener { type T = Box; fn on_release(&self, key: Self::K, value: Self::T) { - self.0.insert_async(key, value); + self.0.insert_if_not_exists_async(key, value); } } @@ -127,7 +128,7 @@ pub struct SstableStore { block_cache: BlockCache, meta_cache: Arc>>, - data_file_cache: FileCache>, + data_file_cache: FileCache, meta_file_cache: FileCache>, recent_filter: Option>>, @@ -140,7 +141,7 @@ impl SstableStore { block_cache_capacity: usize, meta_cache_capacity: usize, high_priority_ratio: usize, - data_file_cache: FileCache>, + data_file_cache: FileCache, meta_file_cache: FileCache>, recent_filter: Option>>, ) -> Self { @@ -284,6 +285,7 @@ impl SstableStore { .await .map_err(HummockError::file_cache)? { + let block = block.into_inner(); return Ok(block); } @@ -323,7 +325,9 @@ impl SstableStore { sst_id: object_id, block_idx: block_index as u64, }, - block.clone(), + CachedBlock::Loaded { + block: block.clone(), + }, ); Ok(BlockResponse::Block(BlockHolder::from_owned_block(block))) } @@ -524,7 +528,7 @@ impl SstableStore { self.recent_filter.as_ref() } - pub fn data_file_cache(&self) -> &FileCache> { + pub fn data_file_cache(&self) -> &FileCache { &self.data_file_cache } } From a962e0afb85c89c3ee2e3a3909e29bc9794b0342 Mon Sep 17 00:00:00 2001 From: Huangjw <1223644280@qq.com> Date: Tue, 14 Nov 2023 00:25:24 +0800 Subject: [PATCH 51/77] fix(docker): fix memory limit suffix in docker compose (#13382) --- docker/docker-compose-standalone.yml | 4 ++-- docker/docker-compose-with-hdfs.yml | 16 ++++++++-------- docker/docker-compose.yml | 16 ++++++++-------- 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/docker/docker-compose-standalone.yml b/docker/docker-compose-standalone.yml index f7dcb9b576ebf..3cac6d250cb72 100644 --- a/docker/docker-compose-standalone.yml +++ b/docker/docker-compose-standalone.yml @@ -75,9 +75,9 @@ services: deploy: resources: limits: - memory: 28Gi + memory: 28G reservations: - memory: 28Gi + memory: 28G etcd-0: image: "quay.io/coreos/etcd:v3.5.7" diff --git a/docker/docker-compose-with-hdfs.yml b/docker/docker-compose-with-hdfs.yml index 0929faf9fadd9..ed7ecd8788465 100644 --- a/docker/docker-compose-with-hdfs.yml +++ b/docker/docker-compose-with-hdfs.yml @@ -38,9 +38,9 @@ services: deploy: resources: limits: - memory: 2Gi + memory: 2G reservations: - memory: 1Gi + memory: 1G compute-node-0: image: "ghcr.io/risingwavelabs/risingwave:RW_1.1_HADOOP2-x86_64" command: @@ -78,9 +78,9 @@ services: deploy: resources: limits: - memory: 26Gi + memory: 26G reservations: - memory: 26Gi + memory: 26G etcd-0: image: "quay.io/coreos/etcd:v3.5.7" command: @@ -167,9 +167,9 @@ services: deploy: resources: limits: - memory: 2Gi + memory: 2G reservations: - memory: 1Gi + memory: 1G grafana-0: image: "grafana/grafana-oss:latest" command: [] @@ -242,9 +242,9 @@ services: deploy: resources: limits: - memory: 2Gi + memory: 2G reservations: - memory: 1Gi + memory: 1G prometheus-0: image: "prom/prometheus:latest" command: diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index 36c67d7235b61..0421a762ee016 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -42,9 +42,9 @@ services: deploy: resources: limits: - memory: 2Gi + memory: 2G reservations: - memory: 1Gi + memory: 1G compute-node-0: <<: *image command: @@ -84,9 +84,9 @@ services: deploy: resources: limits: - memory: 26Gi + memory: 26G reservations: - memory: 26Gi + memory: 26G etcd-0: image: "quay.io/coreos/etcd:v3.5.7" command: @@ -174,9 +174,9 @@ services: deploy: resources: limits: - memory: 2Gi + memory: 2G reservations: - memory: 1Gi + memory: 1G grafana-0: image: "grafana/grafana-oss:latest" command: [] @@ -250,9 +250,9 @@ services: deploy: resources: limits: - memory: 2Gi + memory: 2G reservations: - memory: 1Gi + memory: 1G minio-0: image: "quay.io/minio/minio:latest" command: From e46151bdcb5427ddc7306015d2729a7edda312ce Mon Sep 17 00:00:00 2001 From: lmatz Date: Tue, 14 Nov 2023 01:46:07 +0800 Subject: [PATCH 52/77] chore: add a configuraiton toml for serving-only CN (#13239) --- src/config/serving-only.toml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) create mode 100644 src/config/serving-only.toml diff --git a/src/config/serving-only.toml b/src/config/serving-only.toml new file mode 100644 index 0000000000000..a7ef71a7c9846 --- /dev/null +++ b/src/config/serving-only.toml @@ -0,0 +1,14 @@ +[storage] +# Shared buffer is useless to a serving-only compute node. +shared_buffer_capacity_mb = 1 +# Compactor is irrelevant to a serving-only compute node. +compactor_memory_limit_mb = 1 +# Suppose a serving-only CN has 16GB memory in total, we recommend using 30% by default: 16*0.3=4.8 +block_cache_capacity_mb = 4800 +# Suppose a serving-only CN has 16GB memory in toal, we recommend using 10% by default: 16*0.1=1.6 +meta_cache_capacity_mb = 1600 +# There will be 16GB - 4.8GB - 1.6GB - reserved memory 16GB * 0.2 (recommended and hard coded by developers) = 6.4GB for executing serving queries. We call it "compute memory". + +# If one batch query is resource-intensive and its runtime memory consumption exceeds 6.4GB, it will kill itself automatically before triggering out-of-memory error. The reserved memory is allocated for all types of temporary memory usage and slackness. +# Although we don't recommend users to execute OLAP-style batch queries that take a huge amount of input data, e.g. more than dozens of GB, users can adjust the configuration if such batch query is needed and the default configuration above leaves too few "compute memory". +# Feel free to allocate less memory for "block cache" and "meta cache" to increase "compute memory". From dad71c1f41c23956ac864f9eff9066b0afc8bab3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 14 Nov 2023 10:30:09 +0800 Subject: [PATCH 53/77] chore(deps): Bump fs-err from 2.9.0 to 2.10.0 (#13399) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 7 +++++-- src/prost/Cargo.toml | 2 +- src/risedevtool/Cargo.toml | 2 +- src/risedevtool/config/Cargo.toml | 2 +- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b2928f34902cb..196939aa6036e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3284,9 +3284,12 @@ dependencies = [ [[package]] name = "fs-err" -version = "2.9.0" +version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0845fa252299212f0389d64ba26f34fa32cfe41588355f21ed507c59a0f64541" +checksum = "fb5fd9bcbe8b1087cbd395b51498c01bc997cef73e778a80b77a811af5e2d29f" +dependencies = [ + "autocfg", +] [[package]] name = "fs2" diff --git a/src/prost/Cargo.toml b/src/prost/Cargo.toml index 8c4468d337942..61546fc091eb7 100644 --- a/src/prost/Cargo.toml +++ b/src/prost/Cargo.toml @@ -21,7 +21,7 @@ tonic = { workspace = true } workspace-hack = { path = "../workspace-hack" } [build-dependencies] -fs-err = "2.9" +fs-err = "2.10" pbjson-build = "0.5" tonic-build = { workspace = true } walkdir = "2" diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index b67fa31498c01..e3caf3c7d34da 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -18,7 +18,7 @@ anyhow = { version = "1", features = ["backtrace"] } chrono = { version = "0.4", default-features = false, features = ["clock", "std"] } clap = { version = "4", features = ["derive"] } console = "0.15" -fs-err = "2.9.0" +fs-err = "2.10.0" glob = "0.3" google-cloud-pubsub = "0.20" indicatif = "0.17" diff --git a/src/risedevtool/config/Cargo.toml b/src/risedevtool/config/Cargo.toml index e33eb0afd1647..c0eecd8ce0c46 100644 --- a/src/risedevtool/config/Cargo.toml +++ b/src/risedevtool/config/Cargo.toml @@ -13,7 +13,7 @@ clap = { version = "4", features = ["derive"] } console = "0.15" dialoguer = "0.11" enum-iterator = "1" -fs-err = "2.9.0" +fs-err = "2.10.0" itertools = "0.11" [lints] From b569b926e01a5dd7d6d429bbe09040db904b6cb9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 14 Nov 2023 11:12:07 +0800 Subject: [PATCH 54/77] chore(deps): Bump fancy-regex from 0.11.0 to 0.12.0 (#13398) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- src/expr/impl/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 196939aa6036e..7091b2682c594 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2954,9 +2954,9 @@ checksum = "4443176a9f2c162692bd3d352d745ef9413eec5782a80d8fd6f8a1ac692a07f7" [[package]] name = "fancy-regex" -version = "0.11.0" +version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b95f7c0680e4142284cf8b22c14a476e87d61b004a3a0861872b32ef7ead40a2" +checksum = "7493d4c459da9f84325ad297371a6b2b8a162800873a22e3b6b6512e61d18c05" dependencies = [ "bit-set", "regex", diff --git a/src/expr/impl/Cargo.toml b/src/expr/impl/Cargo.toml index 466c6905212cd..dbc3e60c880ee 100644 --- a/src/expr/impl/Cargo.toml +++ b/src/expr/impl/Cargo.toml @@ -24,7 +24,7 @@ chrono = { version = "0.4", default-features = false, features = [ "clock", "std", ] } -fancy-regex = "0.11" +fancy-regex = "0.12" futures-async-stream = { workspace = true } futures-util = "0.3" hex = "0.4" From 3fd8140905ff92d2aeb768f84957bd3089e468a2 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 14 Nov 2023 11:12:35 +0800 Subject: [PATCH 55/77] fix(connector): warn on undefined nested field in struct when parsing JSON (#13384) Signed-off-by: Bugen Zhao --- e2e_test/udf/udf.slt | 10 ++-- src/common/src/types/mod.rs | 2 +- src/common/src/types/struct_type.rs | 2 +- src/connector/src/parser/json_parser.rs | 50 +++++++++++++++++++ src/connector/src/parser/unified/json.rs | 16 ++++-- .../tests/testdata/output/struct_query.yaml | 4 +- 6 files changed, 70 insertions(+), 14 deletions(-) diff --git a/e2e_test/udf/udf.slt b/e2e_test/udf/udf.slt index aebfd3d33e697..7ebdf49d0b20d 100644 --- a/e2e_test/udf/udf.slt +++ b/e2e_test/udf/udf.slt @@ -72,19 +72,19 @@ query TTTTT rowsort show functions ---- array_access character varying[], integer character varying (empty) http://localhost:8815 -extract_tcp_info bytea struct (empty) http://localhost:8815 +extract_tcp_info bytea struct (empty) http://localhost:8815 gcd integer, integer integer (empty) http://localhost:8815 gcd integer, integer, integer integer (empty) http://localhost:8815 hex_to_dec character varying numeric (empty) http://localhost:8815 int_42 (empty) integer (empty) http://localhost:8815 jsonb_access jsonb, integer jsonb (empty) http://localhost:8815 jsonb_array_identity jsonb[] jsonb[] (empty) http://localhost:8815 -jsonb_array_struct_identity struct struct (empty) http://localhost:8815 +jsonb_array_struct_identity struct struct (empty) http://localhost:8815 jsonb_concat jsonb[] jsonb (empty) http://localhost:8815 -return_all boolean, smallint, integer, bigint, real, double precision, numeric, date, time without time zone, timestamp without time zone, interval, character varying, bytea, jsonb struct (empty) http://localhost:8815 -return_all_arrays boolean[], smallint[], integer[], bigint[], real[], double precision[], numeric[], date[], time without time zone[], timestamp without time zone[], interval[], character varying[], bytea[], jsonb[] struct (empty) http://localhost:8815 +return_all boolean, smallint, integer, bigint, real, double precision, numeric, date, time without time zone, timestamp without time zone, interval, character varying, bytea, jsonb struct (empty) http://localhost:8815 +return_all_arrays boolean[], smallint[], integer[], bigint[], real[], double precision[], numeric[], date[], time without time zone[], timestamp without time zone[], interval[], character varying[], bytea[], jsonb[] struct (empty) http://localhost:8815 series integer integer (empty) http://localhost:8815 -split character varying struct (empty) http://localhost:8815 +split character varying struct (empty) http://localhost:8815 query I select int_42(); diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 386f63280a557..3d62780c19b91 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -1220,7 +1220,7 @@ mod tests { ); assert_eq!( format!("{}", d), - "struct".to_string() + "struct".to_string() ); } diff --git a/src/common/src/types/struct_type.rs b/src/common/src/types/struct_type.rs index 239f506db8267..7aca0e727c82e 100644 --- a/src/common/src/types/struct_type.rs +++ b/src/common/src/types/struct_type.rs @@ -140,7 +140,7 @@ impl Display for StructType { (self.0.field_types.iter()) .zip_eq_fast(self.0.field_names.iter()) .map(|(d, s)| format!("{} {}", s, d)) - .join(",") + .join(", ") ) } } diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 3be0acaa5a392..3922bde228a41 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -515,6 +515,56 @@ mod tests { assert_eq!(row, expected.into()); } + #[cfg(not(madsim))] // Traced test does not work with madsim + #[tokio::test] + #[tracing_test::traced_test] + async fn test_json_parse_struct_missing_field_warning() { + let descs = vec![ColumnDesc::new_struct( + "struct", + 0, + "", + vec![ + ColumnDesc::new_atomic(DataType::Varchar, "varchar", 1), + ColumnDesc::new_atomic(DataType::Boolean, "boolean", 2), + ], + )] + .iter() + .map(SourceColumnDesc::from) + .collect_vec(); + + let parser = JsonParser::new( + SpecificParserConfig::DEFAULT_PLAIN_JSON, + descs.clone(), + Default::default(), + ) + .unwrap(); + let payload = br#" + { + "struct": { + "varchar": "varchar" + } + } + "# + .to_vec(); + let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 1); + { + let writer = builder.row_writer(); + parser.parse_inner(payload, writer).await.unwrap(); + } + let chunk = builder.finish(); + let (op, row) = chunk.rows().next().unwrap(); + assert_eq!(op, Op::Insert); + let row = row.into_owned_row().into_inner(); + + let expected = vec![Some(ScalarImpl::Struct(StructValue::new(vec![ + Some(ScalarImpl::Utf8("varchar".into())), + None, + ])))]; + assert_eq!(row, expected.into()); + + assert!(logs_contain("undefined nested field, padding with `NULL`")); + } + #[tokio::test] async fn test_json_upsert_parser() { let items = [ diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index b770649b052c9..9033cb049a465 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -457,11 +457,17 @@ impl JsonParseOptions { .names() .zip_eq_fast(struct_type_info.types()) .map(|(field_name, field_type)| { - self.parse( - json_object_get_case_insensitive(value, field_name) - .unwrap_or(&BorrowedValue::Static(simd_json::StaticNode::Null)), - Some(field_type), - ) + let field_value = json_object_get_case_insensitive(value, field_name) + .unwrap_or_else(|| { + let error = AccessError::Undefined { + name: field_name.to_owned(), + path: struct_type_info.to_string(), // TODO: this is not good, we should maintain a path stack + }; + // TODO: is it possible to unify the logging with the one in `do_action`? + tracing::warn!(%error, "undefined nested field, padding with `NULL`"); + &BorrowedValue::Static(simd_json::StaticNode::Null) + }); + self.parse(field_value, Some(field_type)) }) .collect::>()?, ) diff --git a/src/frontend/planner_test/tests/testdata/output/struct_query.yaml b/src/frontend/planner_test/tests/testdata/output/struct_query.yaml index c981787d56ba4..525ca3880ad47 100644 --- a/src/frontend/planner_test/tests/testdata/output/struct_query.yaml +++ b/src/frontend/planner_test/tests/testdata/output/struct_query.yaml @@ -300,7 +300,7 @@ Failed to bind expression: avg(country) Caused by: - Invalid input syntax: Invalid aggregation: avg(struct
) + Invalid input syntax: Invalid aggregation: avg(struct
) create_source: format: plain encode: protobuf @@ -422,7 +422,7 @@ - sql: | CREATE TABLE a (c STRUCT, j INTEGER>); INSERT INTO a VALUES (1); - binder_error: 'Bind error: cannot cast type "integer" to "struct,j integer>" in Assign context' + binder_error: 'Bind error: cannot cast type "integer" to "struct, j integer>" in Assign context' - name: test struct type alignment in CASE expression sql: | select CASE WHEN false THEN ROW(0, INTERVAL '1') WHEN true THEN ROW(1.1, INTERVAL '1') ELSE ROW(1, INTERVAL '1') END; From 26be82aace38a3b117de001f8f78914f450945fe Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Tue, 14 Nov 2023 11:18:18 +0800 Subject: [PATCH 56/77] refactor(stream): use `ExecutorParams::info` in source-related executors (#13380) Signed-off-by: Richard Chien --- src/compute/tests/integration_tests.rs | 26 +++++++----- src/stream/src/executor/dml.rs | 35 +++++++--------- src/stream/src/executor/row_id_gen.rs | 31 ++++++-------- .../src/executor/source/fetch_executor.rs | 27 ++++-------- .../src/executor/source/fs_source_executor.rs | 27 ++++-------- .../src/executor/source/list_executor.rs | 23 ++++------- .../src/executor/source/source_executor.rs | 41 ++++++++----------- src/stream/src/from_proto/dml.rs | 8 +--- src/stream/src/from_proto/row_id_gen.rs | 4 +- src/stream/src/from_proto/source/fs_fetch.rs | 22 ++-------- .../src/from_proto/source/trad_source.rs | 34 ++++----------- 11 files changed, 100 insertions(+), 178 deletions(-) diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 078dfa05aa448..cbd2cf85cc124 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -56,7 +56,7 @@ use risingwave_stream::executor::monitor::StreamingMetrics; use risingwave_stream::executor::row_id_gen::RowIdGenExecutor; use risingwave_stream::executor::source_executor::SourceExecutor; use risingwave_stream::executor::{ - ActorContext, Barrier, Executor, MaterializeExecutor, Message, PkIndices, + ActorContext, Barrier, Executor, ExecutorInfo, MaterializeExecutor, Message, PkIndices, }; use tokio::sync::mpsc::unbounded_channel; @@ -171,23 +171,27 @@ async fn test_table_materialize() -> StreamResult<()> { // Create a `SourceExecutor` to read the changes. let source_executor = SourceExecutor::::new( actor_ctx.clone(), - all_schema.clone(), - pk_indices.clone(), + ExecutorInfo { + schema: all_schema.clone(), + pk_indices: pk_indices.clone(), + identity: format!("SourceExecutor {:X}", 1), + }, None, // There is no external stream source. Arc::new(StreamingMetrics::unused()), barrier_rx, system_params_manager.get_params(), - 1, SourceCtrlOpts::default(), ConnectorParams::default(), ); // Create a `DmlExecutor` to accept data change from users. let dml_executor = DmlExecutor::new( + ExecutorInfo { + schema: all_schema.clone(), + pk_indices: pk_indices.clone(), + identity: format!("DmlExecutor {:X}", 2), + }, Box::new(source_executor), - all_schema.clone(), - pk_indices.clone(), - 2, dml_manager.clone(), table_id, INITIAL_TABLE_VERSION_ID, @@ -196,10 +200,12 @@ async fn test_table_materialize() -> StreamResult<()> { let row_id_gen_executor = RowIdGenExecutor::new( actor_ctx, + ExecutorInfo { + schema: all_schema.clone(), + pk_indices: pk_indices.clone(), + identity: format!("RowIdGenExecutor {:X}", 3), + }, Box::new(dml_executor), - all_schema.clone(), - pk_indices.clone(), - 3, row_id_index, vnodes, ); diff --git a/src/stream/src/executor/dml.rs b/src/stream/src/executor/dml.rs index b324f4666cc2e..6daf95ea0b023 100644 --- a/src/stream/src/executor/dml.rs +++ b/src/stream/src/executor/dml.rs @@ -25,21 +25,17 @@ use risingwave_source::dml_manager::DmlManagerRef; use super::error::StreamExecutorError; use super::{ - expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, Message, Mutation, - PkIndices, PkIndicesRef, + expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, + Mutation, PkIndicesRef, }; use crate::executor::stream_reader::StreamReaderWithPause; /// [`DmlExecutor`] accepts both stream data and batch data for data manipulation on a specific /// table. The two streams will be merged into one and then sent to downstream. pub struct DmlExecutor { - upstream: BoxedExecutor, - - schema: Schema, - - pk_indices: PkIndices, + info: ExecutorInfo, - identity: String, + upstream: BoxedExecutor, /// Stores the information of batch data channels. dml_manager: DmlManagerRef, @@ -73,20 +69,16 @@ struct TxnBuffer { impl DmlExecutor { #[allow(clippy::too_many_arguments)] pub fn new( + info: ExecutorInfo, upstream: BoxedExecutor, - schema: Schema, - pk_indices: PkIndices, - executor_id: u64, dml_manager: DmlManagerRef, table_id: TableId, table_version_id: TableVersionId, column_descs: Vec, ) -> Self { Self { + info, upstream, - schema, - pk_indices, - identity: format!("DmlExecutor {:X}", executor_id), dml_manager, table_id, table_version_id, @@ -205,15 +197,15 @@ impl Executor for DmlExecutor { } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - &self.identity + &self.info.identity } } @@ -248,12 +240,15 @@ mod tests { let dml_manager = Arc::new(DmlManager::for_test()); let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone()); + let info = ExecutorInfo { + schema, + pk_indices, + identity: "DmlExecutor".to_string(), + }; let dml_executor = Box::new(DmlExecutor::new( + info, Box::new(source), - schema, - pk_indices, - 1, dml_manager.clone(), table_id, INITIAL_TABLE_VERSION_ID, diff --git a/src/stream/src/executor/row_id_gen.rs b/src/stream/src/executor/row_id_gen.rs index 88a11f03c663b..b3b096d060baa 100644 --- a/src/stream/src/executor/row_id_gen.rs +++ b/src/stream/src/executor/row_id_gen.rs @@ -26,22 +26,17 @@ use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::row_id::RowIdGenerator; use super::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, Executor, PkIndices, PkIndicesRef, + expect_first_barrier, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, PkIndicesRef, }; use crate::executor::{Message, StreamExecutorError}; /// [`RowIdGenExecutor`] generates row id for data, where the user has not specified a pk. pub struct RowIdGenExecutor { ctx: ActorContextRef, + info: ExecutorInfo, upstream: Option, - schema: Schema, - - pk_indices: PkIndices, - - identity: String, - row_id_index: usize, row_id_generator: RowIdGenerator, @@ -50,19 +45,15 @@ pub struct RowIdGenExecutor { impl RowIdGenExecutor { pub fn new( ctx: ActorContextRef, + info: ExecutorInfo, upstream: BoxedExecutor, - schema: Schema, - pk_indices: PkIndices, - executor_id: u64, row_id_index: usize, vnodes: Bitmap, ) -> Self { Self { ctx, + info, upstream: Some(upstream), - schema, - pk_indices, - identity: format!("RowIdGenExecutor {:X}", executor_id), row_id_index, row_id_generator: Self::new_generator(&vnodes), } @@ -130,15 +121,15 @@ impl Executor for RowIdGenExecutor { } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - &self.identity + &self.info.identity } } @@ -166,10 +157,12 @@ mod tests { let (mut tx, upstream) = MockSource::channel(schema.clone(), pk_indices.clone()); let row_id_gen_executor = Box::new(RowIdGenExecutor::new( ActorContext::create(233), + ExecutorInfo { + schema, + pk_indices, + identity: "RowIdGenExecutor".to_string(), + }, Box::new(upstream), - schema, - pk_indices, - 1, row_id_index, row_id_generator, )); diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 3e7ea84bcfcce..c45fda5e7bba3 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -36,8 +36,8 @@ use risingwave_storage::StateStore; use crate::executor::stream_reader::StreamReaderWithPause; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, Message, - Mutation, PkIndices, PkIndicesRef, SourceStateTableHandler, StreamExecutorError, + expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, + ExecutorInfo, Message, Mutation, PkIndicesRef, SourceStateTableHandler, StreamExecutorError, StreamExecutorResult, StreamSourceCore, }; @@ -47,12 +47,7 @@ type SplitBatch = Option>; pub struct FsFetchExecutor { actor_ctx: ActorContextRef, - - identity: String, - - schema: Schema, - - pk_indices: PkIndices, + info: ExecutorInfo, /// Streaming source for external stream_source_core: Option>, @@ -71,19 +66,15 @@ impl FsFetchExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - schema: Schema, - pk_indices: PkIndices, + info: ExecutorInfo, stream_source_core: StreamSourceCore, - executor_id: u64, upstream: BoxedExecutor, source_ctrl_opts: SourceCtrlOpts, connector_params: ConnectorParams, ) -> Self { Self { actor_ctx, - identity: format!("FsFetchExecutor {:X}", executor_id), - schema, - pk_indices, + info, stream_source_core: Some(stream_source_core), upstream: Some(upstream), source_ctrl_opts, @@ -323,15 +314,15 @@ impl Executor for FsFetchExecutor { } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - self.identity.as_str() + &self.info.identity } } @@ -341,7 +332,7 @@ impl Debug for FsFetchExecutor { f.debug_struct("FsFetchExecutor") .field("source_id", &core.source_id) .field("column_ids", &core.column_ids) - .field("pk_indices", &self.pk_indices) + .field("pk_indices", &self.info.pk_indices) .finish() } else { f.debug_struct("FsFetchExecutor").finish() diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index bba0e30eb5712..2a1650d175d4f 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -47,12 +47,7 @@ const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5; /// such as s3. pub struct FsSourceExecutor { actor_ctx: ActorContextRef, - - identity: String, - - schema: Schema, - - pk_indices: PkIndices, + info: ExecutorInfo, /// Streaming source for external stream_source_core: StreamSourceCore, @@ -72,21 +67,17 @@ pub struct FsSourceExecutor { impl FsSourceExecutor { #[allow(clippy::too_many_arguments)] pub fn new( - ctx: ActorContextRef, - schema: Schema, - pk_indices: PkIndices, + actor_ctx: ActorContextRef, + info: ExecutorInfo, stream_source_core: StreamSourceCore, metrics: Arc, barrier_receiver: UnboundedReceiver, system_params: SystemParamsReaderRef, - executor_id: u64, source_ctrl_opts: SourceCtrlOpts, ) -> StreamResult { Ok(Self { - actor_ctx: ctx, - identity: format!("SourceExecutor {:X}", executor_id), - schema, - pk_indices, + actor_ctx, + info, stream_source_core, metrics, barrier_receiver: Some(barrier_receiver), @@ -470,15 +461,15 @@ impl Executor for FsSourceExecutor { } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - self.identity.as_str() + &self.info.identity } } @@ -487,7 +478,7 @@ impl Debug for FsSourceExecutor { f.debug_struct("FsSourceExecutor") .field("source_id", &self.stream_source_core.source_id) .field("column_ids", &self.stream_source_core.column_ids) - .field("pk_indices", &self.pk_indices) + .field("pk_indices", &self.info.pk_indices) .finish() } } diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index 53e8854594ce4..38de2e612d1c7 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -37,12 +37,7 @@ use crate::executor::*; #[allow(dead_code)] pub struct FsListExecutor { actor_ctx: ActorContextRef, - - identity: String, - - schema: Schema, - - pk_indices: PkIndices, + info: ExecutorInfo, /// Streaming source for external stream_source_core: Option>, @@ -67,21 +62,17 @@ impl FsListExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - schema: Schema, - pk_indices: PkIndices, + info: ExecutorInfo, stream_source_core: Option>, metrics: Arc, barrier_receiver: UnboundedReceiver, system_params: SystemParamsReaderRef, - executor_id: u64, source_ctrl_opts: SourceCtrlOpts, connector_params: ConnectorParams, ) -> Self { Self { actor_ctx, - identity: format!("FsListExecutor {:X}", executor_id), - schema, - pk_indices, + info, stream_source_core, metrics, barrier_receiver: Some(barrier_receiver), @@ -204,15 +195,15 @@ impl Executor for FsListExecutor { } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - self.identity.as_str() + &self.info.identity } } @@ -222,7 +213,7 @@ impl Debug for FsListExecutor { f.debug_struct("FsListExecutor") .field("source_id", &core.source_id) .field("column_ids", &core.column_ids) - .field("pk_indices", &self.pk_indices) + .field("pk_indices", &self.info.pk_indices) .finish() } else { f.debug_struct("FsListExecutor").finish() diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 45d4f53c69bea..ecca51d48b8f0 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -42,12 +42,7 @@ const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5; pub struct SourceExecutor { actor_ctx: ActorContextRef, - - identity: String, - - schema: Schema, - - pk_indices: PkIndices, + info: ExecutorInfo, /// Streaming source for external stream_source_core: Option>, @@ -72,21 +67,17 @@ impl SourceExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - schema: Schema, - pk_indices: PkIndices, + info: ExecutorInfo, stream_source_core: Option>, metrics: Arc, barrier_receiver: UnboundedReceiver, system_params: SystemParamsReaderRef, - executor_id: u64, source_ctrl_opts: SourceCtrlOpts, connector_params: ConnectorParams, ) -> Self { Self { actor_ctx, - identity: format!("SourceExecutor {:X}", executor_id), - schema, - pk_indices, + info, stream_source_core, metrics, barrier_receiver: Some(barrier_receiver), @@ -553,7 +544,7 @@ impl SourceExecutor { self_paused = true; tracing::warn!( "source {} paused, wait barrier for {:?}", - self.identity, + self.info.identity, last_barrier_time.elapsed() ); stream.pause_stream(); @@ -652,15 +643,15 @@ impl Executor for SourceExecutor { } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - self.identity.as_str() + &self.info.identity } } @@ -670,7 +661,7 @@ impl Debug for SourceExecutor { f.debug_struct("SourceExecutor") .field("source_id", &core.source_id) .field("column_ids", &core.column_ids) - .field("pk_indices", &self.pk_indices) + .field("pk_indices", &self.info.pk_indices) .finish() } else { f.debug_struct("SourceExecutor").finish() @@ -746,13 +737,15 @@ mod tests { let executor = SourceExecutor::new( ActorContext::create(0), - schema, - pk_indices, + ExecutorInfo { + schema, + pk_indices, + identity: "SourceExecutor".to_string(), + }, Some(core), Arc::new(StreamingMetrics::unused()), barrier_rx, system_params_manager.get_params(), - 1, SourceCtrlOpts::default(), ConnectorParams::default(), ); @@ -838,13 +831,15 @@ mod tests { let executor = SourceExecutor::new( ActorContext::create(0), - schema, - pk_indices, + ExecutorInfo { + schema, + pk_indices, + identity: "SourceExecutor".to_string(), + }, Some(core), Arc::new(StreamingMetrics::unused()), barrier_rx, system_params_manager.get_params(), - 1, SourceCtrlOpts::default(), ConnectorParams::default(), ); diff --git a/src/stream/src/from_proto/dml.rs b/src/stream/src/from_proto/dml.rs index e9242edf98f9f..d0d2576e1fe49 100644 --- a/src/stream/src/from_proto/dml.rs +++ b/src/stream/src/from_proto/dml.rs @@ -13,7 +13,7 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::catalog::{Schema, TableId}; +use risingwave_common::catalog::TableId; use risingwave_pb::stream_plan::DmlNode; use risingwave_storage::StateStore; @@ -37,13 +37,9 @@ impl ExecutorBuilder for DmlExecutorBuilder { let [upstream]: [_; 1] = params.input.try_into().unwrap(); let table_id = TableId::new(node.table_id); let column_descs = node.column_descs.iter().map(Into::into).collect_vec(); - let fields = column_descs.iter().map(Into::into).collect_vec(); - let schema = Schema::new(fields); Ok(Box::new(DmlExecutor::new( + params.info, upstream, - schema, - params.pk_indices, - params.executor_id, params.env.dml_manager_ref(), table_id, node.table_version_id, diff --git a/src/stream/src/from_proto/row_id_gen.rs b/src/stream/src/from_proto/row_id_gen.rs index a54dad43acacf..a76ef081876e5 100644 --- a/src/stream/src/from_proto/row_id_gen.rs +++ b/src/stream/src/from_proto/row_id_gen.rs @@ -39,10 +39,8 @@ impl ExecutorBuilder for RowIdGenExecutorBuilder { .expect("vnodes not set for row id gen executor"); let executor = RowIdGenExecutor::new( params.actor_context, + params.info, upstream, - params.schema, - params.pk_indices, - params.executor_id, node.row_id_index as _, vnodes, ); diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index 5519428d0e46b..a380d5989eace 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -14,8 +14,7 @@ use std::sync::Arc; -use risingwave_common::catalog::{ColumnId, Field, Schema, TableId}; -use risingwave_common::types::DataType; +use risingwave_common::catalog::{ColumnId, TableId}; use risingwave_connector::source::SourceCtrlOpts; use risingwave_pb::stream_plan::StreamFsFetchNode; use risingwave_source::source_desc::SourceDescBuilder; @@ -63,22 +62,11 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { chunk_size: params.env.config().developer.chunk_size, }; - let column_ids: Vec<_> = source + let source_column_ids: Vec<_> = source .columns .iter() .map(|column| ColumnId::from(column.get_column_desc().unwrap().column_id)) .collect(); - let fields = source - .columns - .iter() - .map(|prost| { - let column_desc = prost.column_desc.as_ref().unwrap(); - let data_type = DataType::from(column_desc.column_type.as_ref().unwrap()); - let name = column_desc.name.clone(); - Field::with_name(data_type, name) - }) - .collect(); - let schema = Schema::new(fields); let vnodes = Some(Arc::new( params @@ -94,17 +82,15 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { let stream_source_core = StreamSourceCore::new( source_id, source_name, - column_ids, + source_column_ids, source_desc_builder, state_table_handler, ); let executor = FsFetchExecutor::new( params.actor_context, - schema, - params.pk_indices, + params.info, stream_source_core, - params.executor_id, upstream, source_ctrl_opts, params.env.connector_params(), diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index c719a75bc3b4b..54c7f16797c0b 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::{ColumnId, Field, Schema, TableId}; -use risingwave_common::types::DataType; +use risingwave_common::catalog::{ColumnId, Schema, TableId}; use risingwave_common::util::sort_util::OrderType; use risingwave_connector::source::external::{CdcTableType, SchemaTableName}; use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; @@ -78,22 +77,11 @@ impl ExecutorBuilder for SourceExecutorBuilder { chunk_size: params.env.config().developer.chunk_size, }; - let column_ids: Vec<_> = source + let source_column_ids: Vec<_> = source .columns .iter() .map(|column| ColumnId::from(column.get_column_desc().unwrap().column_id)) .collect(); - let fields = source - .columns - .iter() - .map(|prost| { - let column_desc = prost.column_desc.as_ref().unwrap(); - let data_type = DataType::from(column_desc.column_type.as_ref().unwrap()); - let name = column_desc.name.clone(); - Field::with_name(data_type, name) - }) - .collect(); - let schema = Schema::new(fields); let state_table_handler = SourceStateTableHandler::from_table_catalog( source.state_table.as_ref().unwrap(), @@ -103,7 +91,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { let stream_source_core = StreamSourceCore::new( source_id, source_name, - column_ids, + source_column_ids, source_desc_builder, state_table_handler, ); @@ -120,26 +108,22 @@ impl ExecutorBuilder for SourceExecutorBuilder { if is_fs_connector { FsSourceExecutor::new( params.actor_context, - schema, - params.pk_indices, + params.info, stream_source_core, params.executor_stats, barrier_receiver, system_params, - params.executor_id, source_ctrl_opts, )? .boxed() } else if is_fs_v2_connector { FsListExecutor::new( params.actor_context.clone(), - schema.clone(), - params.pk_indices.clone(), + params.info.clone(), Some(stream_source_core), params.executor_stats.clone(), barrier_receiver, system_params, - params.executor_id, source_ctrl_opts.clone(), params.env.connector_params(), ) @@ -147,13 +131,11 @@ impl ExecutorBuilder for SourceExecutorBuilder { } else { let source_exec = SourceExecutor::new( params.actor_context.clone(), - schema.clone(), - params.pk_indices.clone(), + params.info.clone(), Some(stream_source_core), params.executor_stats.clone(), barrier_receiver, system_params, - params.executor_id, source_ctrl_opts.clone(), params.env.connector_params(), ); @@ -218,13 +200,11 @@ impl ExecutorBuilder for SourceExecutorBuilder { // `PanicStateStore` type here for indication. Ok(SourceExecutor::::new( params.actor_context, - params.schema, - params.pk_indices, + params.info, None, params.executor_stats, barrier_receiver, system_params, - params.executor_id, // we don't expect any data in, so no need to set chunk_sizes SourceCtrlOpts::default(), params.env.connector_params(), From 69ed22719e0a15572b407636aa0ad83864e2786d Mon Sep 17 00:00:00 2001 From: Dylan Date: Tue, 14 Nov 2023 12:28:43 +0800 Subject: [PATCH 57/77] fix(scale): fix serving fragment bitmap for scale (#13395) --- src/meta/src/serving/mod.rs | 6 ++--- src/meta/src/stream/scale.rs | 48 ++++++++++++++++++++++++++++++++++++ 2 files changed, 51 insertions(+), 3 deletions(-) diff --git a/src/meta/src/serving/mod.rs b/src/meta/src/serving/mod.rs index 521a8b9ad1c0d..a027348cc9619 100644 --- a/src/meta/src/serving/mod.rs +++ b/src/meta/src/serving/mod.rs @@ -43,7 +43,7 @@ impl ServingVnodeMapping { /// Upsert mapping for given fragments according to the latest `workers`. /// Returns (successful updates, failed updates). - fn upsert( + pub fn upsert( &self, streaming_parallelisms: HashMap, workers: &[WorkerNode], @@ -79,7 +79,7 @@ impl ServingVnodeMapping { } } -fn to_fragment_parallel_unit_mapping( +pub(crate) fn to_fragment_parallel_unit_mapping( mappings: &HashMap, ) -> Vec { mappings @@ -91,7 +91,7 @@ fn to_fragment_parallel_unit_mapping( .collect() } -fn to_deleted_fragment_parallel_unit_mapping( +pub(crate) fn to_deleted_fragment_parallel_unit_mapping( fragment_ids: &[FragmentId], ) -> Vec { fragment_ids diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 99a4d6fc76e92..9656b2409226c 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -27,9 +27,11 @@ use risingwave_common::hash::{ActorMapping, ParallelUnitId, VirtualNode}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_pb::common::{ActorInfo, ParallelUnit, WorkerNode}; use risingwave_pb::meta::get_reschedule_plan_request::{Policy, StableResizePolicy}; +use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::table_fragments::actor_status::ActorState; use risingwave_pb::meta::table_fragments::fragment::FragmentDistributionType; use risingwave_pb::meta::table_fragments::{self, ActorStatus, Fragment}; +use risingwave_pb::meta::FragmentParallelUnitMappings; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DispatcherType, FragmentTypeFlag, StreamActor, StreamNode}; use risingwave_pb::stream_service::{ @@ -40,6 +42,10 @@ use uuid::Uuid; use crate::barrier::Reschedule; use crate::manager::{ClusterManagerRef, FragmentManagerRef, IdCategory, MetaSrvEnv, WorkerId}; use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments}; +use crate::serving::{ + to_deleted_fragment_parallel_unit_mapping, to_fragment_parallel_unit_mapping, + ServingVnodeMapping, +}; use crate::storage::{MetaStore, MetaStoreError, MetaStoreRef, Transaction, DEFAULT_COLUMN_FAMILY}; use crate::stream::SourceManagerRef; use crate::{MetaError, MetaResult}; @@ -1532,6 +1538,48 @@ impl ScaleController { .post_apply_reschedules(reschedules.clone()) .await?; + // Update serving fragment info after rescheduling in meta store. + if !reschedules.is_empty() { + let workers = self + .cluster_manager + .list_active_serving_compute_nodes() + .await; + let streaming_parallelisms = self + .fragment_manager + .running_fragment_parallelisms(Some(reschedules.keys().cloned().collect())) + .await; + let serving_vnode_mapping = Arc::new(ServingVnodeMapping::default()); + let (upserted, failed) = serving_vnode_mapping.upsert(streaming_parallelisms, &workers); + if !upserted.is_empty() { + tracing::debug!( + "Update serving vnode mapping for fragments {:?}.", + upserted.keys() + ); + self.env + .notification_manager() + .notify_frontend_without_version( + Operation::Update, + Info::ServingParallelUnitMappings(FragmentParallelUnitMappings { + mappings: to_fragment_parallel_unit_mapping(&upserted), + }), + ); + } + if !failed.is_empty() { + tracing::debug!( + "Fail to update serving vnode mapping for fragments {:?}.", + failed + ); + self.env + .notification_manager() + .notify_frontend_without_version( + Operation::Delete, + Info::ServingParallelUnitMappings(FragmentParallelUnitMappings { + mappings: to_deleted_fragment_parallel_unit_mapping(&failed), + }), + ); + } + } + let mut stream_source_actor_splits = HashMap::new(); let mut stream_source_dropped_actors = HashSet::new(); From 4a5a3311a6565cb3015b459ff149c35c5608f451 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Tue, 14 Nov 2023 12:54:32 +0800 Subject: [PATCH 58/77] feat(storage): support max version pinning duration (#13340) --- src/common/src/config.rs | 7 +++++ src/config/example.toml | 1 + src/meta/src/hummock/manager/mod.rs | 8 ++++- .../hummock/local_version/pinned_version.rs | 31 ++++++++++++++----- .../src/hummock/store/hummock_storage.rs | 1 + src/storage/src/opts.rs | 2 ++ 6 files changed, 42 insertions(+), 8 deletions(-) diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 02b97bf6e618c..ddb4ed8ec1c47 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -552,6 +552,9 @@ pub struct StorageConfig { #[serde(default = "default::storage::max_preload_wait_time_mill")] pub max_preload_wait_time_mill: u64, + #[serde(default = "default::storage::max_version_pinning_duration_sec")] + pub max_version_pinning_duration_sec: u64, + #[serde(default = "default::storage::object_store_streaming_read_timeout_ms")] pub object_store_streaming_read_timeout_ms: u64, #[serde(default = "default::storage::object_store_streaming_upload_timeout_ms")] @@ -1067,6 +1070,10 @@ pub mod default { 0 } + pub fn max_version_pinning_duration_sec() -> u64 { + 3 * 3600 + } + pub fn object_store_streaming_read_timeout_ms() -> u64 { 10 * 60 * 1000 } diff --git a/src/config/example.toml b/src/config/example.toml index 8e8a2ac0bb406..c5c9bb3aef021 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -98,6 +98,7 @@ min_sst_size_for_streaming_upload = 33554432 max_sub_compaction = 4 max_concurrent_compaction_task_number = 16 max_preload_wait_time_mill = 0 +max_version_pinning_duration_sec = 10800 object_store_streaming_read_timeout_ms = 600000 object_store_streaming_upload_timeout_ms = 600000 object_store_upload_timeout_ms = 3600000 diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index a578cb571c706..5a3ba0384a52c 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -628,7 +628,7 @@ impl HummockManager { /// Unpin all pins which belongs to `context_id` and has an id which is older than /// `unpin_before`. All versions >= `unpin_before` will be treated as if they are all pinned by - /// this `context_id` so they will not be vacummed. + /// this `context_id` so they will not be vacuumed. #[named] pub async fn unpin_version_before( &self, @@ -646,6 +646,12 @@ impl HummockManager { min_pinned_id: 0, }, ); + assert!( + context_pinned_version.min_pinned_id <= unpin_before, + "val must be monotonically non-decreasing. old = {}, new = {}.", + context_pinned_version.min_pinned_id, + unpin_before + ); context_pinned_version.min_pinned_id = unpin_before; commit_multi_var!( self, diff --git a/src/storage/src/hummock/local_version/pinned_version.rs b/src/storage/src/hummock/local_version/pinned_version.rs index 636533d35403f..876a2f9a1f6e0 100644 --- a/src/storage/src/hummock/local_version/pinned_version.rs +++ b/src/storage/src/hummock/local_version/pinned_version.rs @@ -16,7 +16,7 @@ use std::collections::{BTreeMap, HashMap}; use std::iter::empty; use std::ops::Deref; use std::sync::Arc; -use std::time::Duration; +use std::time::{Duration, Instant}; use auto_enums::auto_enum; use risingwave_common::catalog::TableId; @@ -167,6 +167,7 @@ impl PinnedVersion { pub(crate) async fn start_pinned_version_worker( mut rx: UnboundedReceiver, hummock_meta_client: Arc, + max_version_pinning_duration_sec: u64, ) { let min_execute_interval = Duration::from_millis(1000); let max_retry_interval = Duration::from_secs(10); @@ -180,21 +181,34 @@ pub(crate) async fn start_pinned_version_worker( min_execute_interval_tick.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); let mut need_unpin = false; - let mut version_ids_in_use: BTreeMap = BTreeMap::new(); - + let mut version_ids_in_use: BTreeMap = BTreeMap::new(); + let max_version_pinning_duration_sec = Duration::from_secs(max_version_pinning_duration_sec); // For each run in the loop, accumulate versions to unpin and call unpin RPC once. loop { min_execute_interval_tick.tick().await; + // 0. Expire versions. + while version_ids_in_use.len() > 1 && let Some(e) = version_ids_in_use.first_entry() { + if e.get().1.elapsed() < max_version_pinning_duration_sec { + break; + } + need_unpin = true; + e.remove(); + } + // 1. Collect new versions to unpin. let mut versions_to_unpin = vec![]; + let inst = Instant::now(); 'collect: loop { match rx.try_recv() { Ok(version_action) => match version_action { PinVersionAction::Pin(version_id) => { version_ids_in_use .entry(version_id) - .and_modify(|counter| *counter += 1) - .or_insert(1); + .and_modify(|e| { + e.0 += 1; + e.1 = inst; + }) + .or_insert((1, inst)); } PinVersionAction::Unpin(version_id) => { versions_to_unpin.push(version_id); @@ -220,13 +234,16 @@ pub(crate) async fn start_pinned_version_worker( for version in &versions_to_unpin { match version_ids_in_use.get_mut(version) { - Some(counter) => { + Some((counter, _)) => { *counter -= 1; if *counter == 0 { version_ids_in_use.remove(version); } } - None => tracing::warn!("version {} to unpin dose not exist", version), + None => tracing::warn!( + "version {} to unpin does not exist, may already be unpinned due to expiration", + version + ), } } diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index e0125263df0a5..16aa9d474de0a 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -156,6 +156,7 @@ impl HummockStorage { tokio::spawn(start_pinned_version_worker( pin_version_rx, hummock_meta_client.clone(), + options.max_version_pinning_duration_sec, )); let filter_key_extractor_manager = FilterKeyExtractorManager::RpcFilterKeyExtractorManager( filter_key_extractor_manager.clone(), diff --git a/src/storage/src/opts.rs b/src/storage/src/opts.rs index 16112d1ae9f24..2fb02e61512c0 100644 --- a/src/storage/src/opts.rs +++ b/src/storage/src/opts.rs @@ -63,6 +63,7 @@ pub struct StorageOpts { /// Max sub compaction task numbers pub max_sub_compaction: u32, pub max_concurrent_compaction_task_number: u64, + pub max_version_pinning_duration_sec: u64, pub data_file_cache_dir: String, pub data_file_cache_capacity_mb: usize, @@ -164,6 +165,7 @@ impl From<(&RwConfig, &SystemParamsReader, &StorageMemoryConfig)> for StorageOpt min_sst_size_for_streaming_upload: c.storage.min_sst_size_for_streaming_upload, max_sub_compaction: c.storage.max_sub_compaction, max_concurrent_compaction_task_number: c.storage.max_concurrent_compaction_task_number, + max_version_pinning_duration_sec: c.storage.max_version_pinning_duration_sec, data_file_cache_dir: c.storage.data_file_cache.dir.clone(), data_file_cache_capacity_mb: c.storage.data_file_cache.capacity_mb, data_file_cache_file_capacity_mb: c.storage.data_file_cache.file_capacity_mb, From e1b003d9604cffd43bebcc6c3f48b296f479ddc4 Mon Sep 17 00:00:00 2001 From: Tao Wu Date: Tue, 14 Nov 2023 13:08:09 +0800 Subject: [PATCH 59/77] test: add a demo for cockroach sink (#13308) --- ci/workflows/integration-tests.yml | 5 ++ integration_tests/cockroach-sink/README.md | 16 ++++ .../cockroach-sink/create_mv.sql | 18 ++++ .../cockroach-sink/create_source.sql | 55 +++++++++++++ integration_tests/cockroach-sink/data_check | 1 + .../cockroach-sink/docker-compose.yml | 82 +++++++++++++++++++ .../cockroach-sink/postgres_prepare.sql | 25 ++++++ integration_tests/cockroach-sink/query.sql | 6 ++ 8 files changed, 208 insertions(+) create mode 100644 integration_tests/cockroach-sink/README.md create mode 100644 integration_tests/cockroach-sink/create_mv.sql create mode 100644 integration_tests/cockroach-sink/create_source.sql create mode 100644 integration_tests/cockroach-sink/data_check create mode 100644 integration_tests/cockroach-sink/docker-compose.yml create mode 100644 integration_tests/cockroach-sink/postgres_prepare.sql create mode 100644 integration_tests/cockroach-sink/query.sql diff --git a/ci/workflows/integration-tests.yml b/ci/workflows/integration-tests.yml index 455f29b210ec1..b17ea6ea14919 100644 --- a/ci/workflows/integration-tests.yml +++ b/ci/workflows/integration-tests.yml @@ -113,6 +113,7 @@ steps: - "citus-cdc" - "kinesis-s3-source" - "clickhouse-sink" + - "cockroach-sink" format: - "json" - "protobuf" @@ -149,3 +150,7 @@ steps: testcase: "clickhouse-sink" format: "protobuf" skip: true + - with: + testcase: "cockroach-sink" + format: "protobuf" + skip: true diff --git a/integration_tests/cockroach-sink/README.md b/integration_tests/cockroach-sink/README.md new file mode 100644 index 0000000000000..5792c08021be8 --- /dev/null +++ b/integration_tests/cockroach-sink/README.md @@ -0,0 +1,16 @@ +# HOW-TO + +This demo showcases how to sink RisingWave's data to an external CockroachDB. A data loader is included in the docker compose. Once the cluster starts, it continuously ingests data into a Kafka topic `user_behaviors`. + +During CI, the integration test will: + +1. Run `docker compose up -d` and start the cluster. +2. After 20-30s, run `create_source.sql`. +3. After 10s, run `create_mv.sql`. +4. After another 10s, the tester will check if the ingestion is successful by creating a materialized view upon the source. It also checks if the MV created in the 3rd step has persisted the data. + +To connect to the Postgres outside the container via psql: + +```sh +psql -h localhost -p 26257 -d defaultdb -U root +``` diff --git a/integration_tests/cockroach-sink/create_mv.sql b/integration_tests/cockroach-sink/create_mv.sql new file mode 100644 index 0000000000000..29fdfa5cfdc4c --- /dev/null +++ b/integration_tests/cockroach-sink/create_mv.sql @@ -0,0 +1,18 @@ +CREATE MATERIALIZED VIEW target_count AS +SELECT + target_id, + COUNT(*) AS target_count +FROM + user_behaviors +GROUP BY + target_id; + +CREATE SINK target_count_postgres_sink +FROM + target_count WITH ( + connector = 'jdbc', + jdbc.url = 'jdbc:postgresql://cockroachdb:26257/defaultdb?user=root', + table.name = 'target_count', + type = 'upsert', + primary_key = 'target_id' + ); diff --git a/integration_tests/cockroach-sink/create_source.sql b/integration_tests/cockroach-sink/create_source.sql new file mode 100644 index 0000000000000..b37504e75dcce --- /dev/null +++ b/integration_tests/cockroach-sink/create_source.sql @@ -0,0 +1,55 @@ +CREATE SOURCE user_behaviors ( + user_id VARCHAR, + target_id VARCHAR, + target_type VARCHAR, + event_timestamp TIMESTAMPTZ, + behavior_type VARCHAR, + parent_target_type VARCHAR, + parent_target_id VARCHAR +) WITH ( + connector = 'kafka', + topic = 'user_behaviors', + properties.bootstrap.server = 'message_queue:29092', + scan.startup.mode = 'earliest' +) FORMAT PLAIN ENCODE JSON; + +CREATE TABLE data_types ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + timestamptz_column TIMESTAMPTZ, + interval_column INTERVAL, + jsonb_column JSONB, + bytea_column BYTEA, + array_column VARCHAR[] +); + +-- sink data_type table to pg +CREATE SINK data_types_postgres_sink +FROM + data_types WITH ( + connector = 'jdbc', + jdbc.url = 'jdbc:postgresql://cockroachdb:26257/defaultdb?user=root', + table.name = 'data_types', + type='upsert', + primary_key = 'id' +); + +INSERT INTO data_types (id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, timestamptz_column, interval_column, jsonb_column, bytea_column, array_column) +VALUES + (1, 'Varchar value 1', 'Text value 1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '2023-05-22 12:34:56+00:00', '1 day', '{"key": "value"}', E'\\xDEADBEEF', ARRAY['Value 1', 'Value 2']), + (2, 'Varchar value 2', 'Text value 2', 234, 567, 890, 23.45, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2023-05-23 23:45:01+00:00', '2 days', '{"key": "value2"}', E'\\xFEEDBEEF', ARRAY['Value 3', 'Value 4']), + (3, 'Varchar value 3', 'Text value 3', 345, 678, 901, 34.56, 78.90, 12.34, TRUE, '2023-05-24', '12:34:56', '2023-05-24 12:34:56', '2023-05-24 12:34:56+00:00', '3 days', '{"key": "value3"}', E'\\xCAFEBABE', ARRAY['Value 5', 'Value 6']), + (4, 'Varchar value 4', 'Text value 4', 456, 789, 012, 45.67, 89.01, 23.45, FALSE, '2023-05-25', '23:45:01', '2023-05-25 23:45:01', '2023-05-25 23:45:01+00:00', '4 days', '{"key": "value4"}', E'\\xBABEC0DE', ARRAY['Value 7', 'Value 8']), + (5, 'Varchar value 5', 'Text value 5', 567, 890, 123, 56.78, 90.12, 34.56, TRUE, '2023-05-26', '12:34:56', '2023-05-26 12:34:56', '2023-05-26 12:34:56+00:00', '5 days', '{"key": "value5"}', E'\\xDEADBABE', ARRAY['Value 9', 'Value 10']); + diff --git a/integration_tests/cockroach-sink/data_check b/integration_tests/cockroach-sink/data_check new file mode 100644 index 0000000000000..3835eb979b86e --- /dev/null +++ b/integration_tests/cockroach-sink/data_check @@ -0,0 +1 @@ +user_behaviors,target_count \ No newline at end of file diff --git a/integration_tests/cockroach-sink/docker-compose.yml b/integration_tests/cockroach-sink/docker-compose.yml new file mode 100644 index 0000000000000..295f07170adb2 --- /dev/null +++ b/integration_tests/cockroach-sink/docker-compose.yml @@ -0,0 +1,82 @@ +--- +version: "3" +services: + compactor-0: + extends: + file: ../../docker/docker-compose.yml + service: compactor-0 + compute-node-0: + extends: + file: ../../docker/docker-compose.yml + service: compute-node-0 + etcd-0: + extends: + file: ../../docker/docker-compose.yml + service: etcd-0 + frontend-node-0: + extends: + file: ../../docker/docker-compose.yml + service: frontend-node-0 + grafana-0: + extends: + file: ../../docker/docker-compose.yml + service: grafana-0 + meta-node-0: + extends: + file: ../../docker/docker-compose.yml + service: meta-node-0 + minio-0: + extends: + file: ../../docker/docker-compose.yml + service: minio-0 + prometheus-0: + extends: + file: ../../docker/docker-compose.yml + service: prometheus-0 + message_queue: + extends: + file: ../../docker/docker-compose.yml + service: message_queue + datagen: + build: ../datagen + depends_on: [message_queue] + command: + - /bin/sh + - -c + - /datagen --mode clickstream --qps 2 kafka --brokers message_queue:29092 + restart: always + container_name: datagen + cockroachdb: + image: cockroachdb/cockroach:v23.1.11 + command: start-single-node --insecure + ports: + - "26257:26257" # CockroachDB default port + - "8080:8080" # CockroachDB Web UI port + restart: always + container_name: cockroachdb + prepare_postgres: + image: postgres + depends_on: + - cockroachdb + command: + - /bin/sh + - -c + - "psql postgresql://root@cockroachdb:26257/defaultdb < postgres_prepare.sql" + volumes: + - "./postgres_prepare.sql:/postgres_prepare.sql" + container_name: prepare_postgres + restart: on-failure +volumes: + compute-node-0: + external: false + etcd-0: + external: false + grafana-0: + external: false + minio-0: + external: false + prometheus-0: + external: false + message_queue: + external: false +name: risingwave-compose diff --git a/integration_tests/cockroach-sink/postgres_prepare.sql b/integration_tests/cockroach-sink/postgres_prepare.sql new file mode 100644 index 0000000000000..595cf960121d7 --- /dev/null +++ b/integration_tests/cockroach-sink/postgres_prepare.sql @@ -0,0 +1,25 @@ +CREATE TABLE target_count ( + target_id VARCHAR(128) primary key, + target_count BIGINT +); + +CREATE TABLE data_types ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + timestamptz_column TIMESTAMPTZ, + interval_column INTERVAL, + jsonb_column JSONB, + bytea_column BYTEA, + array_column VARCHAR[] +); \ No newline at end of file diff --git a/integration_tests/cockroach-sink/query.sql b/integration_tests/cockroach-sink/query.sql new file mode 100644 index 0000000000000..e09c66a255f10 --- /dev/null +++ b/integration_tests/cockroach-sink/query.sql @@ -0,0 +1,6 @@ +SELECT + * +FROM + target_count +LIMIT + 10; \ No newline at end of file From 932d694b6b2e41a6ceb70cc93e0af50e2ebffa8b Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 14 Nov 2023 13:25:28 +0800 Subject: [PATCH 60/77] fix(streaming): avoid shadowing identity when flow controlled (#13405) Signed-off-by: Bugen Zhao --- src/stream/src/executor/flow_control.rs | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/stream/src/executor/flow_control.rs b/src/stream/src/executor/flow_control.rs index a96518d35ad75..cf21375da3cba 100644 --- a/src/stream/src/executor/flow_control.rs +++ b/src/stream/src/executor/flow_control.rs @@ -32,12 +32,22 @@ use super::*; /// It is used to throttle problematic MVs that are consuming too much resources. pub struct FlowControlExecutor { input: BoxedExecutor, + identity: String, rate_limit: Option, } impl FlowControlExecutor { pub fn new(input: Box, rate_limit: Option) -> Self { - Self { input, rate_limit } + let identity = if rate_limit.is_some() { + format!("{} (flow controlled)", input.identity()) + } else { + input.identity().to_owned() + }; + Self { + input, + identity, + rate_limit, + } } #[try_stream(ok = Message, error = StreamExecutorError)] @@ -97,6 +107,6 @@ impl Executor for FlowControlExecutor { } fn identity(&self) -> &str { - "FlowControlExecutor" + &self.identity } } From 0c348c5bb1b4fb08607b852812778de77deee649 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 14 Nov 2023 13:39:15 +0800 Subject: [PATCH 61/77] fix(cdc): fix some cdc backfill issues (#13397) --- src/connector/src/source/external.rs | 55 ++++++++----------- .../tests/testdata/output/create_source.yaml | 5 +- .../plan_node/stream_cdc_table_scan.rs | 2 +- src/sqlparser/README.md | 4 ++ src/sqlparser/src/ast/mod.rs | 2 +- src/sqlparser/tests/testdata/create.yaml | 4 ++ .../src/executor/backfill/cdc/cdc_backfill.rs | 3 + .../src/from_proto/source/trad_source.rs | 2 +- src/stream/src/from_proto/stream_scan.rs | 5 +- 9 files changed, 44 insertions(+), 38 deletions(-) diff --git a/src/connector/src/source/external.rs b/src/connector/src/source/external.rs index ea17a0a2272a0..c70f41b32b581 100644 --- a/src/connector/src/source/external.rs +++ b/src/connector/src/source/external.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::collections::HashMap; -use std::future::Future; use anyhow::anyhow; use futures::stream::BoxStream; @@ -62,14 +61,14 @@ impl CdcTableType { matches!(self, Self::MySql) } - pub fn create_table_reader( + pub async fn create_table_reader( &self, properties: HashMap, schema: Schema, ) -> ConnectorResult { match self { Self::MySql => Ok(ExternalTableReaderImpl::MySql( - MySqlExternalTableReader::new(properties, schema)?, + MySqlExternalTableReader::new(properties, schema).await?, )), _ => bail!(ConnectorError::Config(anyhow!( "invalid external table type: {:?}", @@ -212,7 +211,7 @@ impl MySqlOffset { pub trait ExternalTableReader { fn get_normalized_table_name(&self, table_name: &SchemaTableName) -> String; - fn current_cdc_offset(&self) -> impl Future> + Send + '_; + async fn current_cdc_offset(&self) -> ConnectorResult; fn parse_binlog_offset(&self, offset: &str) -> ConnectorResult; @@ -232,10 +231,11 @@ pub enum ExternalTableReaderImpl { #[derive(Debug)] pub struct MySqlExternalTableReader { - pool: mysql_async::Pool, config: ExternalTableConfig, rw_schema: Schema, field_names: String, + // use mutex to provide shared mutable access to the connection + conn: tokio::sync::Mutex, } #[derive(Debug, Clone, Deserialize)] @@ -260,11 +260,7 @@ impl ExternalTableReader for MySqlExternalTableReader { } async fn current_cdc_offset(&self) -> ConnectorResult { - let mut conn = self - .pool - .get_conn() - .await - .map_err(|e| ConnectorError::Connection(anyhow!(e)))?; + let mut conn = self.conn.lock().await; let sql = "SHOW MASTER STATUS".to_string(); let mut rs = conn.query::(sql).await?; @@ -294,7 +290,10 @@ impl ExternalTableReader for MySqlExternalTableReader { } impl MySqlExternalTableReader { - pub fn new(properties: HashMap, rw_schema: Schema) -> ConnectorResult { + pub async fn new( + properties: HashMap, + rw_schema: Schema, + ) -> ConnectorResult { tracing::debug!(?rw_schema, "create mysql external table reader"); let config = serde_json::from_value::( @@ -308,7 +307,8 @@ impl MySqlExternalTableReader { "mysql://{}:{}@{}:{}/{}", config.username, config.password, config.host, config.port, config.database ); - let pool = mysql_async::Pool::from_url(database_url)?; + let opts = mysql_async::Opts::from_url(&database_url).map_err(mysql_async::Error::Url)?; + let conn = mysql_async::Conn::new(opts).await?; let field_names = rw_schema .fields @@ -318,18 +318,13 @@ impl MySqlExternalTableReader { .join(","); Ok(Self { - pool, config, rw_schema, field_names, + conn: tokio::sync::Mutex::new(conn), }) } - pub async fn disconnect(&self) -> ConnectorResult<()> { - self.pool.clone().disconnect().await?; - Ok(()) - } - #[try_stream(boxed, ok = OwnedRow, error = ConnectorError)] async fn snapshot_read_inner( &self, @@ -356,11 +351,7 @@ impl MySqlExternalTableReader { ) }; - let mut conn = self - .pool - .get_conn() - .await - .map_err(|e| ConnectorError::Connection(anyhow!(e)))?; + let mut conn = self.conn.lock().await; // Set session timezone to UTC conn.exec_drop("SET time_zone = \"+00:00\"", ()).await?; @@ -428,7 +419,7 @@ impl MySqlExternalTableReader { let rs_stream = sql .with(Params::from(params)) - .stream::(&mut conn) + .stream::(&mut *conn) .await?; let row_stream = rs_stream.map(|row| { @@ -587,10 +578,10 @@ mod tests { sink_id: Default::default(), properties: Default::default(), columns: vec![ - ColumnDesc::unnamed(ColumnId::new(1), DataType::Int32), - ColumnDesc::unnamed(ColumnId::new(2), DataType::Decimal), - ColumnDesc::unnamed(ColumnId::new(3), DataType::Varchar), - ColumnDesc::unnamed(ColumnId::new(4), DataType::Date), + ColumnDesc::named("v1".into(), ColumnId::new(1), DataType::Int32), + ColumnDesc::named("v2".into(), ColumnId::new(2), DataType::Decimal), + ColumnDesc::named("v3".into(), ColumnId::new(3), DataType::Varchar), + ColumnDesc::named("v4".into(), ColumnId::new(4), DataType::Date), ], downstream_pk: vec![0], sink_type: SinkType::AppendOnly, @@ -605,15 +596,17 @@ mod tests { "port" => "8306", "username" => "root", "password" => "123456", - "database.name" => "mydb", + "database.name" => "mytest", "table.name" => "t1")); - let reader = MySqlExternalTableReader::new(props, rw_schema).unwrap(); + let reader = MySqlExternalTableReader::new(props, rw_schema) + .await + .unwrap(); let offset = reader.current_cdc_offset().await.unwrap(); println!("BinlogOffset: {:?}", offset); let table_name = SchemaTableName { - schema_name: "mydb".to_string(), + schema_name: "mytest".to_string(), table_name: "t1".to_string(), }; diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index fef47aa736aff..1801fa33d75ea 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -57,7 +57,8 @@ ) from mysql_mydb table 'mydb.t1'; explain_output: | StreamMaterialize { columns: [v1, v2], stream_key: [v1], pk_columns: [v1], pk_conflict: Overwrite } - └─StreamDml { columns: [v1, v2] } - └─StreamCdcTableScan { table: mydb.t1, columns: [v1, v2] } + └─StreamExchange { dist: HashShard(mydb.t1.v1) } + └─StreamDml { columns: [v1, v2] } + └─StreamCdcTableScan { table: mydb.t1, columns: [v1, v2] } with_config_map: CDC_BACKFILL: 'true' diff --git a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs index 07284b6cf5436..7b68e9d80c93e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs @@ -45,7 +45,7 @@ pub struct StreamCdcTableScan { impl StreamCdcTableScan { pub fn new(core: generic::Scan) -> Self { let batch_plan_id = core.ctx.next_plan_node_id(); - let distribution = Distribution::Single; + let distribution = Distribution::SomeShard; let base = PlanBase::new_stream_with_core( &core, distribution, diff --git a/src/sqlparser/README.md b/src/sqlparser/README.md index a9d7f341ef11b..20a5ac0c8e6ab 100644 --- a/src/sqlparser/README.md +++ b/src/sqlparser/README.md @@ -2,3 +2,7 @@ This parser is a fork of . + +## Add a new test case +1. Copy an item in the yaml file and edit the `input` to the sql you want to test +2. Run `./risedev do-apply-parser-test` to regenerate the `formatted_sql` whicih is the expected output \ No newline at end of file diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 757fb7f2a237b..560403ac6e918 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1573,7 +1573,7 @@ impl fmt::Display for Statement { } if let Some(info) = cdc_table_info { write!(f, " FROM {}", info.source_name)?; - write!(f, " TABLE {}", info.external_table_name)?; + write!(f, " TABLE '{}'", info.external_table_name)?; } Ok(()) } diff --git a/src/sqlparser/tests/testdata/create.yaml b/src/sqlparser/tests/testdata/create.yaml index cde40d8a75da1..ff5cb9678687d 100644 --- a/src/sqlparser/tests/testdata/create.yaml +++ b/src/sqlparser/tests/testdata/create.yaml @@ -19,6 +19,10 @@ error_msg: 'sql parser error: expect description of the format' - input: CREATE SOURCE src FORMAT PLAIN ENCODE JSON formatted_sql: CREATE SOURCE src FORMAT PLAIN ENCODE JSON +- input: CREATE SOURCE mysql_src with ( connector = 'mysql-cdc', hostname = 'localhost', port = '3306', database.name = 'mytest', server.id = '5601' ) + formatted_sql: CREATE SOURCE mysql_src WITH (connector = 'mysql-cdc', hostname = 'localhost', port = '3306', database.name = 'mytest', server.id = '5601') FORMAT PLAIN ENCODE JSON +- input: CREATE TABLE sbtest10 (id INT PRIMARY KEY, k INT, c CHARACTER VARYING, pad CHARACTER VARYING) FROM sbtest TABLE 'mydb.sbtest10' + formatted_sql: CREATE TABLE sbtest10 (id INT PRIMARY KEY, k INT, c CHARACTER VARYING, pad CHARACTER VARYING) FROM sbtest TABLE 'mydb.sbtest10' - input: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://') formatted_sql: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://') formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(ConnectorSchema { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "location", quote_style: None }]), value: SingleQuotedString("file://") }] }), source_watermarks: [] } }' diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index 8b796e538f21a..b756d48944a03 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -446,6 +446,9 @@ impl CdcBackfillExecutor { state_impl.mutate_state(last_binlog_offset).await?; } + // drop reader to release db connection + drop(upstream_table_reader); + tracing::info!( actor = self.actor_id, "CdcBackfill has already finished and forward messages directly to the downstream" diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 54c7f16797c0b..995a83b8f38fd 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -158,7 +158,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { .collect_vec(); let table_reader = table_type - .create_table_reader(source.properties.clone(), table_schema.clone())?; + .create_table_reader(source.properties.clone(), table_schema.clone()).await?; let external_table = ExternalStorageTable::new( TableId::new(source.source_id), upstream_table_name, diff --git a/src/stream/src/from_proto/stream_scan.rs b/src/stream/src/from_proto/stream_scan.rs index 3f002dd81a9cc..0cb9b64bafbf5 100644 --- a/src/stream/src/from_proto/stream_scan.rs +++ b/src/stream/src/from_proto/stream_scan.rs @@ -76,8 +76,9 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { .map(|(k, v)| (k.clone(), v.clone())) .collect(); let table_type = CdcTableType::from_properties(&properties); - let table_reader = - table_type.create_table_reader(properties.clone(), table_schema.clone())?; + let table_reader = table_type + .create_table_reader(properties.clone(), table_schema.clone()) + .await?; let table_pk_order_types = table_desc .pk From 0020507edbc4010b20aeeb560c7bea9159315602 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 14 Nov 2023 00:43:11 -0500 Subject: [PATCH 62/77] refactor(streaming): pass order_col_indices and order_types in minput (#13376) --- proto/stream_plan.proto | 1 + .../src/optimizer/plan_node/generic/agg.rs | 23 +-- .../src/executor/aggregation/agg_group.rs | 5 - .../src/executor/aggregation/agg_state.rs | 29 ++-- src/stream/src/executor/aggregation/minput.rs | 133 +++++++++--------- src/stream/src/executor/hash_agg.rs | 10 +- src/stream/src/executor/simple_agg.rs | 7 +- src/stream/src/executor/test_utils.rs | 20 +-- src/stream/src/from_proto/agg_common.rs | 12 +- 9 files changed, 121 insertions(+), 119 deletions(-) diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 2cd8380e3bcf2..706149b26978b 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -258,6 +258,7 @@ message AggCallState { // for constructing state table column mapping repeated uint32 included_upstream_indices = 2; repeated uint32 table_value_indices = 3; + repeated common.ColumnOrder order_columns = 4; } oneof inner { diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 18ac5525182f3..c309ad9f752e9 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -248,6 +248,11 @@ impl AggCallState { .into_iter() .map(|x| x as _) .collect(), + order_columns: s + .order_columns + .into_iter() + .map(|x| x.to_protobuf()) + .collect(), }, ) } @@ -260,6 +265,7 @@ pub struct MaterializedInputState { pub table: TableCatalog, pub included_upstream_indices: Vec, pub table_value_indices: Vec, + pub order_columns: Vec, } impl Agg { @@ -355,32 +361,30 @@ impl Agg { self.create_table_builder(me.ctx(), window_col_idx); let read_prefix_len_hint = table_builder.get_current_pk_len(); + let mut order_columns = vec![]; let mut table_value_indices = BTreeSet::new(); // table column indices of value columns let mut add_column = - |upstream_idx, order_type, is_value, table_builder: &mut TableCatalogBuilder| { + |upstream_idx, order_type, table_builder: &mut TableCatalogBuilder| { column_mapping.entry(upstream_idx).or_insert_with(|| { let table_col_idx = table_builder.add_column(&in_fields[upstream_idx]); if let Some(order_type) = order_type { table_builder.add_order_column(table_col_idx, order_type); + order_columns.push(ColumnOrder::new(upstream_idx, order_type)); } included_upstream_indices.push(upstream_idx); table_col_idx }); - if is_value { - // note that some indices may be added before as group keys which are not - // value - table_value_indices.insert(column_mapping[&upstream_idx]); - } + table_value_indices.insert(column_mapping[&upstream_idx]); }; for (order_type, idx) in sort_keys { - add_column(idx, Some(order_type), true, &mut table_builder); + add_column(idx, Some(order_type), &mut table_builder); } for idx in extra_keys { - add_column(idx, Some(OrderType::ascending()), true, &mut table_builder); + add_column(idx, Some(OrderType::ascending()), &mut table_builder); } for idx in include_keys { - add_column(idx, None, true, &mut table_builder); + add_column(idx, None, &mut table_builder); } let mapping = @@ -398,6 +402,7 @@ impl Agg { table: table_builder.build(tb_dist.unwrap_or_default(), read_prefix_len_hint), included_upstream_indices, table_value_indices, + order_columns, } }; diff --git a/src/stream/src/executor/aggregation/agg_group.rs b/src/stream/src/executor/aggregation/agg_group.rs index d0e97cd4783e9..aea954f2095a2 100644 --- a/src/stream/src/executor/aggregation/agg_group.rs +++ b/src/stream/src/executor/aggregation/agg_group.rs @@ -31,7 +31,6 @@ use risingwave_storage::StateStore; use super::agg_state::{AggState, AggStateStorage}; use crate::common::table::state_table::StateTable; use crate::executor::error::StreamExecutorResult; -use crate::executor::PkIndices; pub trait Strategy { /// Infer the change type of the aggregation result. Don't need to take the ownership of @@ -199,7 +198,6 @@ impl AggGroup { agg_funcs: &[BoxedAggregateFunction], storages: &[AggStateStorage], intermediate_state_table: &StateTable, - pk_indices: &PkIndices, row_count_index: usize, extreme_cache_size: usize, input_schema: &Schema, @@ -219,7 +217,6 @@ impl AggGroup { agg_func, &storages[idx], encoded_states.as_ref().map(|outputs| &outputs[idx]), - pk_indices, extreme_cache_size, input_schema, )?; @@ -251,7 +248,6 @@ impl AggGroup { agg_funcs: &[BoxedAggregateFunction], storages: &[AggStateStorage], encoded_states: &OwnedRow, - pk_indices: &PkIndices, row_count_index: usize, extreme_cache_size: usize, input_schema: &Schema, @@ -264,7 +260,6 @@ impl AggGroup { agg_func, &storages[idx], Some(&encoded_states[idx]), - pk_indices, extreme_cache_size, input_schema, )?; diff --git a/src/stream/src/executor/aggregation/agg_state.rs b/src/stream/src/executor/aggregation/agg_state.rs index a0413ed4491d2..4b74587c3d8b8 100644 --- a/src/stream/src/executor/aggregation/agg_state.rs +++ b/src/stream/src/executor/aggregation/agg_state.rs @@ -18,6 +18,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::estimate_size::EstimateSize; use risingwave_common::must_match; use risingwave_common::types::Datum; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_expr::aggregate::{AggCall, AggregateState, BoxedAggregateFunction}; use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; @@ -26,7 +27,7 @@ use super::minput::MaterializedInputState; use super::GroupKey; use crate::common::table::state_table::StateTable; use crate::common::StateTableColumnMapping; -use crate::executor::{PkIndices, StreamExecutorResult}; +use crate::executor::StreamExecutorResult; /// Represents the persistent storage of aggregation state. pub enum AggStateStorage { @@ -35,10 +36,11 @@ pub enum AggStateStorage { /// The state is stored as a materialization of input chunks, in a standalone state table. /// `mapping` describes the mapping between the columns in the state table and the input - /// chunks. + /// chunks. `order_columns` list the index and order type of sort keys. MaterializedInput { table: StateTable, mapping: StateTableColumnMapping, + order_columns: Vec, }, } @@ -71,7 +73,6 @@ impl AggState { agg_func: &BoxedAggregateFunction, storage: &AggStateStorage, encoded_state: Option<&Datum>, - pk_indices: &PkIndices, extreme_cache_size: usize, input_schema: &Schema, ) -> StreamExecutorResult { @@ -83,16 +84,18 @@ impl AggState { }; Self::Value(state) } - AggStateStorage::MaterializedInput { mapping, .. } => { - Self::MaterializedInput(Box::new(MaterializedInputState::new( - version, - agg_call, - pk_indices, - mapping, - extreme_cache_size, - input_schema, - )?)) - } + AggStateStorage::MaterializedInput { + mapping, + order_columns, + .. + } => Self::MaterializedInput(Box::new(MaterializedInputState::new( + version, + agg_call, + order_columns, + mapping, + extreme_cache_size, + input_schema, + )?)), }) } diff --git a/src/stream/src/executor/aggregation/minput.rs b/src/stream/src/executor/aggregation/minput.rs index 2392e53d7d49a..096dfad9cb474 100644 --- a/src/stream/src/executor/aggregation/minput.rs +++ b/src/stream/src/executor/aggregation/minput.rs @@ -23,7 +23,7 @@ use risingwave_common::estimate_size::EstimateSize; use risingwave_common::row::{OwnedRow, RowExt}; use risingwave_common::types::Datum; use risingwave_common::util::row_serde::OrderedRowSerde; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_expr::aggregate::{AggCall, AggKind, BoxedAggregateFunction}; use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::store::PrefetchOptions; @@ -34,7 +34,7 @@ use super::GroupKey; use crate::common::cache::{OrderedStateCache, TopNStateCache}; use crate::common::table::state_table::StateTable; use crate::common::StateTableColumnMapping; -use crate::executor::{PkIndices, StreamExecutorResult}; +use crate::executor::StreamExecutorResult; /// Aggregation state as a materialization of input chunks. /// @@ -71,58 +71,23 @@ impl MaterializedInputState { pub fn new( version: PbAggNodeVersion, agg_call: &AggCall, - pk_indices: &PkIndices, + order_columns: &Vec, col_mapping: &StateTableColumnMapping, extreme_cache_size: usize, input_schema: &Schema, ) -> StreamExecutorResult { let arg_col_indices = agg_call.args.val_indices().to_vec(); - let (mut order_col_indices, mut order_types) = - if matches!(agg_call.kind, AggKind::Min | AggKind::Max) { - // `min`/`max` need not to order by any other columns, but have to - // order by the agg value implicitly. - let order_type = if agg_call.kind == AggKind::Min { - OrderType::ascending() - } else { - OrderType::descending() - }; - (vec![arg_col_indices[0]], vec![order_type]) - } else { - agg_call - .column_orders - .iter() - .map(|p| { - ( - p.column_index, - if agg_call.kind == AggKind::LastValue { - p.order_type.reverse() - } else { - p.order_type - }, - ) - }) - .unzip() - }; - - if agg_call.distinct { - if version < PbAggNodeVersion::Issue12140 { - panic!( - "RisingWave versions before issue #12140 is resolved has critical bug, you must re-create current MV to ensure correctness." - ); - } + let mut order_col_indices = vec![]; + let mut order_types = vec![]; + for o in order_columns { + order_col_indices.push(o.column_index); + order_types.push(o.order_type); + } - // If distinct, we need to materialize input with the distinct keys - // As we only support single-column distinct for now, we use the - // `agg_call.args.val_indices()[0]` as the distinct key. - if !order_col_indices.contains(&agg_call.args.val_indices()[0]) { - order_col_indices.push(agg_call.args.val_indices()[0]); - order_types.push(OrderType::ascending()); - } - } else { - // If not distinct, we need to materialize input with the primary keys - let pk_len = pk_indices.len(); - order_col_indices.extend(pk_indices.iter()); - order_types.extend(itertools::repeat_n(OrderType::ascending(), pk_len)); + if agg_call.distinct && version < PbAggNodeVersion::Issue12140 { + panic!( + "RisingWave versions before issue #12140 is resolved has critical bug, you must re-create current MV to ensure correctness." + ); } // map argument columns to state table column indices @@ -272,7 +237,7 @@ mod tests { use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::epoch::EpochPair; - use risingwave_common::util::sort_util::OrderType; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::aggregate::{build_append_only, AggCall}; use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::memory::MemoryStateStore; @@ -325,7 +290,6 @@ mod tests { // Assumption of input schema: // (a: varchar, b: int32, c: int32, _row_id: int64) - let input_pk_indices = vec![3]; // _row_id let field1 = Field::unnamed(DataType::Varchar); let field2 = Field::unnamed(DataType::Int32); let field3 = Field::unnamed(DataType::Int32); @@ -346,10 +310,14 @@ mod tests { ) .await; + let order_columns = vec![ + ColumnOrder::new(2, OrderType::ascending()), // c ASC for AggKind::Min + ColumnOrder::new(3, OrderType::ascending()), // _row_id + ]; let mut state = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call, - &input_pk_indices, + &order_columns, &mapping, usize::MAX, &input_schema, @@ -402,7 +370,7 @@ mod tests { let mut state = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call, - &input_pk_indices, + &order_columns, &mapping, usize::MAX, &input_schema, @@ -420,7 +388,6 @@ mod tests { // Assumption of input schema: // (a: varchar, b: int32, c: int32, _row_id: int64) - let input_pk_indices = vec![3]; // _row_id let field1 = Field::unnamed(DataType::Varchar); let field2 = Field::unnamed(DataType::Int32); let field3 = Field::unnamed(DataType::Int32); @@ -441,10 +408,14 @@ mod tests { ) .await; + let order_columns = vec![ + ColumnOrder::new(2, OrderType::descending()), // c DESC for AggKind::Max + ColumnOrder::new(3, OrderType::ascending()), // _row_id + ]; let mut state = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call, - &input_pk_indices, + &order_columns, &mapping, usize::MAX, &input_schema, @@ -497,7 +468,7 @@ mod tests { let mut state = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call, - &input_pk_indices, + &order_columns, &mapping, usize::MAX, &input_schema, @@ -516,7 +487,6 @@ mod tests { // Assumption of input schema: // (a: varchar, b: int32, c: int32, _row_id: int64) - let input_pk_indices = vec![3]; // _row_id let field1 = Field::unnamed(DataType::Varchar); let field2 = Field::unnamed(DataType::Int32); let field3 = Field::unnamed(DataType::Int32); @@ -552,20 +522,28 @@ mod tests { table_1.init_epoch(epoch); table_2.init_epoch(epoch); + let order_columns_1 = vec![ + ColumnOrder::new(0, OrderType::ascending()), // a ASC for AggKind::Min + ColumnOrder::new(3, OrderType::ascending()), // _row_id + ]; let mut state_1 = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call_1, - &input_pk_indices, + &order_columns_1, &mapping_1, usize::MAX, &input_schema, ) .unwrap(); + let order_columns_2 = vec![ + ColumnOrder::new(1, OrderType::descending()), // b DESC for AggKind::Max + ColumnOrder::new(3, OrderType::ascending()), // _row_id + ]; let mut state_2 = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call_2, - &input_pk_indices, + &order_columns_2, &mapping_2, usize::MAX, &input_schema, @@ -624,7 +602,6 @@ mod tests { // Assumption of input schema: // (a: varchar, b: int32, c: int32, _row_id: int64) - let input_pk_indices = vec![3]; let field1 = Field::unnamed(DataType::Varchar); let field2 = Field::unnamed(DataType::Int32); let field3 = Field::unnamed(DataType::Int32); @@ -646,10 +623,14 @@ mod tests { ) .await; + let order_columns = vec![ + ColumnOrder::new(1, OrderType::descending()), // b DESC for AggKind::Max + ColumnOrder::new(3, OrderType::ascending()), // _row_id + ]; let mut state = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call, - &input_pk_indices, + &order_columns, &mapping, usize::MAX, &input_schema, @@ -701,7 +682,7 @@ mod tests { let mut state = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call, - &input_pk_indices, + &order_columns, &mapping, usize::MAX, &input_schema, @@ -720,7 +701,6 @@ mod tests { // Assumption of input schema: // (a: int32, _row_id: int64) - let input_pk_indices = vec![1]; // _row_id let field1 = Field::unnamed(DataType::Int32); let field2 = Field::unnamed(DataType::Int64); let input_schema = Schema::new(vec![field1, field2]); @@ -742,10 +722,14 @@ mod tests { let mut epoch = EpochPair::new_test_epoch(1); table.init_epoch(epoch); + let order_columns = vec![ + ColumnOrder::new(0, OrderType::ascending()), // a ASC for AggKind::Min + ColumnOrder::new(1, OrderType::ascending()), // _row_id + ]; let mut state = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call, - &input_pk_indices, + &order_columns, &mapping, 1024, &input_schema, @@ -823,7 +807,6 @@ mod tests { // Assumption of input schema: // (a: int32, _row_id: int64) - let input_pk_indices = vec![1]; // _row_id let field1 = Field::unnamed(DataType::Int32); let field2 = Field::unnamed(DataType::Int64); let input_schema = Schema::new(vec![field1, field2]); @@ -842,10 +825,14 @@ mod tests { ) .await; + let order_columns = vec![ + ColumnOrder::new(0, OrderType::ascending()), // a ASC for AggKind::Min + ColumnOrder::new(1, OrderType::ascending()), // _row_id + ]; let mut state = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call, - &input_pk_indices, + &order_columns, &mapping, 3, // cache capacity = 3 for easy testing &input_schema, @@ -924,7 +911,6 @@ mod tests { // (a: varchar, _delim: varchar, b: int32, c: int32, _row_id: int64) // where `a` is the column to aggregate - let input_pk_indices = vec![4]; let input_schema = Schema::new(vec![ Field::unnamed(DataType::Varchar), Field::unnamed(DataType::Varchar), @@ -950,10 +936,15 @@ mod tests { ) .await; + let order_columns = vec![ + ColumnOrder::new(2, OrderType::ascending()), // b ASC + ColumnOrder::new(0, OrderType::descending()), // a DESC + ColumnOrder::new(4, OrderType::ascending()), // _row_id ASC + ]; let mut state = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call, - &input_pk_indices, + &order_columns, &mapping, usize::MAX, &input_schema, @@ -1008,7 +999,6 @@ mod tests { // (a: varchar, b: int32, c: int32, _row_id: int64) // where `a` is the column to aggregate - let input_pk_indices = vec![3]; let field1 = Field::unnamed(DataType::Varchar); let field2 = Field::unnamed(DataType::Int32); let field3 = Field::unnamed(DataType::Int32); @@ -1030,10 +1020,15 @@ mod tests { ) .await; + let order_columns = vec![ + ColumnOrder::new(2, OrderType::ascending()), // c ASC + ColumnOrder::new(0, OrderType::descending()), // a DESC + ColumnOrder::new(3, OrderType::ascending()), // _row_id ASC + ]; let mut state = MaterializedInputState::new( PbAggNodeVersion::Max, &agg_call, - &input_pk_indices, + &order_columns, &mapping, usize::MAX, &input_schema, diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 9bdfc48047338..e2fad54cbb2d0 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -88,9 +88,6 @@ struct ExecutorInner { actor_ctx: ActorContextRef, info: ExecutorInfo, - /// Pk indices from input. - input_pk_indices: Vec, - /// Schema from input. input_schema: Schema, @@ -235,7 +232,6 @@ impl HashAggExecutor { version: args.version, actor_ctx: args.actor_ctx, info: args.info, - input_pk_indices: input_info.pk_indices, input_schema: input_info.schema, group_key_indices: args.extra.group_key_indices, group_key_table_pk_projection: group_key_table_pk_projection.to_vec().into(), @@ -323,7 +319,6 @@ impl HashAggExecutor { &this.agg_funcs, &this.storages, &this.intermediate_state_table, - &this.input_pk_indices, this.row_count_index, this.extreme_cache_size, &this.input_schema, @@ -375,7 +370,7 @@ impl HashAggExecutor { .zip_eq_fast(&mut this.storages) .zip_eq_fast(call_visibilities.iter()) { - if let AggStateStorage::MaterializedInput { table, mapping } = storage + if let AggStateStorage::MaterializedInput { table, mapping, .. } = storage && !call.distinct { let chunk = chunk.project_with_vis(mapping.upstream_columns(), visibility.clone()); @@ -406,7 +401,7 @@ impl HashAggExecutor { .zip_eq_fast(&mut this.storages) .zip_eq_fast(visibilities.iter()) { - if let AggStateStorage::MaterializedInput { table, mapping } = storage + if let AggStateStorage::MaterializedInput { table, mapping, .. } = storage && call.distinct { let chunk = @@ -477,7 +472,6 @@ impl HashAggExecutor { &this.agg_funcs, &this.storages, &states, - &this.input_pk_indices, this.row_count_index, this.extreme_cache_size, &this.input_schema, diff --git a/src/stream/src/executor/simple_agg.rs b/src/stream/src/executor/simple_agg.rs index 081b66a04bedc..e3dc8f842922b 100644 --- a/src/stream/src/executor/simple_agg.rs +++ b/src/stream/src/executor/simple_agg.rs @@ -59,9 +59,6 @@ struct ExecutorInner { actor_ctx: ActorContextRef, info: ExecutorInfo, - /// Pk indices from input. - input_pk_indices: Vec, - /// Schema from input. input_schema: Schema, @@ -141,7 +138,6 @@ impl SimpleAggExecutor { version: args.version, actor_ctx: args.actor_ctx, info: args.info, - input_pk_indices: input_info.pk_indices, input_schema: input_info.schema, agg_funcs: args.agg_calls.iter().map(build_retractable).try_collect()?, agg_calls: args.agg_calls, @@ -188,7 +184,7 @@ impl SimpleAggExecutor { // Materialize input chunk if needed and possible. for (storage, visibility) in this.storages.iter_mut().zip_eq_fast(visibilities.iter()) { - if let AggStateStorage::MaterializedInput { table, mapping } = storage { + if let AggStateStorage::MaterializedInput { table, mapping, .. } = storage { let chunk = chunk.project_with_vis(mapping.upstream_columns(), visibility.clone()); table.write_chunk(chunk); } @@ -279,7 +275,6 @@ impl SimpleAggExecutor { &this.agg_funcs, &this.storages, &this.intermediate_state_table, - &this.input_pk_indices, this.row_count_index, this.extreme_cache_size, &this.input_schema, diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index 3b206c0eb1997..9e5f7ed036b19 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -282,7 +282,7 @@ pub mod agg_executor { use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema, TableId}; use risingwave_common::hash::SerializedKey; use risingwave_common::types::DataType; - use risingwave_common::util::sort_util::OrderType; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::aggregate::{AggCall, AggKind}; use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; @@ -341,30 +341,34 @@ pub mod agg_executor { let mut column_descs = Vec::new(); let mut order_types = Vec::new(); let mut upstream_columns = Vec::new(); + let mut order_columns = Vec::new(); let mut next_column_id = 0; - let mut add_column = |upstream_idx: usize, data_type: DataType, order_type: OrderType| { + let mut add_column = |upstream_idx: usize, data_type: DataType, order_type: Option| { upstream_columns.push(upstream_idx); column_descs.push(ColumnDesc::unnamed( ColumnId::new(next_column_id), data_type, )); + if let Some(order_type) = order_type { + order_columns.push(ColumnOrder::new(upstream_idx as _, order_type)); + order_types.push(order_type); + } next_column_id += 1; - order_types.push(order_type); }; for idx in group_key_indices { - add_column(*idx, input_fields[*idx].data_type(), OrderType::ascending()); + add_column(*idx, input_fields[*idx].data_type(), None); } add_column(agg_call.args.val_indices()[0], agg_call.args.arg_types()[0].clone(), if agg_call.kind == AggKind::Max { - OrderType::descending() + Some(OrderType::descending()) } else { - OrderType::ascending() + Some(OrderType::ascending()) }); for idx in pk_indices { - add_column(*idx, input_fields[*idx].data_type(), OrderType::ascending()); + add_column(*idx, input_fields[*idx].data_type(), Some(OrderType::ascending())); } let state_table = StateTable::new_without_distribution( @@ -375,7 +379,7 @@ pub mod agg_executor { (0..order_types.len()).collect(), ).await; - AggStateStorage::MaterializedInput { table: state_table, mapping: StateTableColumnMapping::new(upstream_columns, None) } + AggStateStorage::MaterializedInput { table: state_table, mapping: StateTableColumnMapping::new(upstream_columns, None), order_columns } } AggKind::Min /* append only */ | AggKind::Max /* append only */ diff --git a/src/stream/src/from_proto/agg_common.rs b/src/stream/src/from_proto/agg_common.rs index 5ae5759aed6f4..33e9e6519affe 100644 --- a/src/stream/src/from_proto/agg_common.rs +++ b/src/stream/src/from_proto/agg_common.rs @@ -16,6 +16,7 @@ use std::collections::HashMap; use std::sync::Arc; use risingwave_common::buffer::Bitmap; +use risingwave_common::util::sort_util::ColumnOrder; use super::*; use crate::common::table::state_table::StateTable; @@ -56,7 +57,16 @@ pub async fn build_agg_state_storages_from_proto( .collect(), ), ); - AggStateStorage::MaterializedInput { table, mapping } + let order_columns = state + .order_columns + .iter() + .map(ColumnOrder::from_protobuf) + .collect(); + AggStateStorage::MaterializedInput { + table, + mapping, + order_columns, + } } }; From 3997e2683e858ac65daca52c5a8ab6c24301206b Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Tue, 14 Nov 2023 14:01:32 +0800 Subject: [PATCH 63/77] refactor: use `ExecutorParams::info` in join related executors (#13389) Signed-off-by: Richard Chien --- src/stream/src/executor/dynamic_filter.rs | 36 +++++---- src/stream/src/executor/hash_join.rs | 87 ++++++++++----------- src/stream/src/executor/lookup.rs | 18 ++--- src/stream/src/executor/lookup/impl_.rs | 29 ++----- src/stream/src/executor/lookup/tests.rs | 39 +++++---- src/stream/src/executor/lookup_union.rs | 2 +- src/stream/src/from_proto/dynamic_filter.rs | 6 +- src/stream/src/from_proto/hash_join.rs | 14 +--- src/stream/src/from_proto/lookup.rs | 3 +- 9 files changed, 103 insertions(+), 131 deletions(-) diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index 36418a20724c3..9d98df4cb820e 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -39,7 +39,8 @@ use super::barrier_align::*; use super::error::StreamExecutorError; use super::monitor::StreamingMetrics; use super::{ - ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, Message, PkIndices, PkIndicesRef, + ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, + PkIndicesRef, }; use crate::common::table::state_table::{StateTable, WatermarkCacheParameterizedStateTable}; use crate::common::StreamChunkBuilder; @@ -48,15 +49,14 @@ use crate::task::ActorEvalErrorReport; pub struct DynamicFilterExecutor { ctx: ActorContextRef, + info: ExecutorInfo, + source_l: Option, source_r: Option, key_l: usize, - pk_indices: PkIndices, - identity: String, comparator: ExprNodeType, left_table: WatermarkCacheParameterizedStateTable, right_table: StateTable, - schema: Schema, metrics: Arc, /// The maximum size of the chunk produced by executor at a time. chunk_size: usize, @@ -66,30 +66,26 @@ impl DynamicFilterExecutor, state_table_r: StateTable, metrics: Arc, chunk_size: usize, ) -> Self { - let schema = source_l.schema().clone(); Self { ctx, + info, source_l: Some(source_l), source_r: Some(source_r), key_l, - pk_indices, - identity: format!("DynamicFilterExecutor {:X}", executor_id), comparator, left_table: state_table_l, right_table: state_table_r, metrics, - schema, chunk_size, } } @@ -265,7 +261,7 @@ impl DynamicFilterExecutor DynamicFilterExecutor Executor } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - self.identity.as_str() + &self.info.identity } } @@ -544,13 +540,19 @@ mod tests { let (tx_l, source_l) = MockSource::channel(schema.clone(), vec![0]); let (tx_r, source_r) = MockSource::channel(schema, vec![]); + let schema = source_l.schema().clone(); + let info = ExecutorInfo { + schema, + pk_indices: vec![0], + identity: "DynamicFilterExecutor".to_string(), + }; + let executor = DynamicFilterExecutor::::new( ActorContext::create(123), + info, Box::new(source_l), Box::new(source_r), 0, - vec![0], - 1, comparator, mem_state_l, mem_state_r, diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 5dd886d9f7009..5ec76328302bf 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -40,8 +40,8 @@ use super::managed_state::join::*; use super::monitor::StreamingMetrics; use super::watermark::*; use super::{ - ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, Message, PkIndices, PkIndicesRef, - Watermark, + ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, + PkIndicesRef, Watermark, }; use crate::common::table::state_table::StateTable; use crate::common::JoinStreamChunkBuilder; @@ -226,6 +226,7 @@ impl JoinSide { /// The output columns are the concatenation of left and right columns. pub struct HashJoinExecutor { ctx: ActorContextRef, + info: ExecutorInfo, /// Left input executor input_l: Option, @@ -233,10 +234,6 @@ pub struct HashJoinExecutor, /// The data types of the formed new columns actual_output_data_types: Vec, - /// The schema of the hash join executor - schema: Schema, - /// The primary key indices of the schema - pk_indices: PkIndices, /// The parameters of the left join executor side_l: JoinSide, /// The parameters of the right join executor @@ -249,12 +246,6 @@ pub struct HashJoinExecutor>, - /// Identity string - identity: String, - - #[expect(dead_code)] - /// Logical Operator Info - op_info: String, /// Whether the logic can be optimized for append-only stream append_only_optimize: bool, @@ -279,8 +270,8 @@ impl std::fmt::Debug .field("input_right", &self.input_r.as_ref().unwrap().identity()) .field("side_l", &self.side_l) .field("side_r", &self.side_r) - .field("pk_indices", &self.pk_indices) - .field("schema", &self.schema) + .field("pk_indices", &self.info.pk_indices) + .field("schema", &self.info.schema) .field("actual_output_data_types", &self.actual_output_data_types) .finish() } @@ -292,15 +283,15 @@ impl Executor for HashJoi } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - self.identity.as_str() + &self.info.identity } } @@ -440,17 +431,15 @@ impl HashJoinExecutor, - pk_indices: PkIndices, output_indices: Vec, - executor_id: u64, cond: Option, inequality_pairs: Vec<(usize, usize, bool, Option)>, - op_info: String, state_table_l: StateTable, degree_state_table_l: StateTable, state_table_r: StateTable, @@ -529,14 +518,6 @@ impl HashJoinExecutor HashJoinExecutor HashJoinExecutor HashJoinExecutor input_watermark.val = value.unwrap(), Err(err) => { if !matches!(err, ExprError::NumericOutOfRange) { - self.ctx.on_compute_error(err, self.identity.as_str()); + self.ctx.on_compute_error(err, &self.info.identity); } continue; } @@ -1384,25 +1362,32 @@ mod tests { ) .await; - let schema_len = match T { - JoinType::LeftSemi | JoinType::LeftAnti => source_l.schema().len(), - JoinType::RightSemi | JoinType::RightAnti => source_r.schema().len(), - _ => source_l.schema().len() + source_r.schema().len(), + let schema = match T { + JoinType::LeftSemi | JoinType::LeftAnti => source_l.schema().clone(), + JoinType::RightSemi | JoinType::RightAnti => source_r.schema().clone(), + _ => [source_l.schema().fields(), source_r.schema().fields()] + .concat() + .into_iter() + .collect(), + }; + let schema_len = schema.len(); + let info = ExecutorInfo { + schema, + pk_indices: vec![1], + identity: "HashJoinExecutor".to_string(), }; let executor = HashJoinExecutor::::new( ActorContext::create(123), + info, Box::new(source_l), Box::new(source_r), params_l, params_r, vec![null_safe], - vec![1], (0..schema_len).collect_vec(), - 1, cond, inequality_pairs, - "HashJoinExecutor".to_string(), state_l, degree_state_l, state_r, @@ -1466,25 +1451,33 @@ mod tests { 0, ) .await; - let schema_len = match T { - JoinType::LeftSemi | JoinType::LeftAnti => source_l.schema().len(), - JoinType::RightSemi | JoinType::RightAnti => source_r.schema().len(), - _ => source_l.schema().len() + source_r.schema().len(), + + let schema = match T { + JoinType::LeftSemi | JoinType::LeftAnti => source_l.schema().clone(), + JoinType::RightSemi | JoinType::RightAnti => source_r.schema().clone(), + _ => [source_l.schema().fields(), source_r.schema().fields()] + .concat() + .into_iter() + .collect(), + }; + let schema_len = schema.len(); + let info = ExecutorInfo { + schema, + pk_indices: vec![1], + identity: "HashJoinExecutor".to_string(), }; let executor = HashJoinExecutor::::new( ActorContext::create(123), + info, Box::new(source_l), Box::new(source_r), params_l, params_r, vec![false], - vec![1], (0..schema_len).collect_vec(), - 1, cond, vec![], - "HashJoinExecutor".to_string(), state_l, degree_state_l, state_r, diff --git a/src/stream/src/executor/lookup.rs b/src/stream/src/executor/lookup.rs index 94e9eeab76742..54dc8beb7c9be 100644 --- a/src/stream/src/executor/lookup.rs +++ b/src/stream/src/executor/lookup.rs @@ -18,7 +18,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::types::DataType; use risingwave_storage::StateStore; -use crate::executor::{Barrier, BoxedMessageStream, Executor, PkIndices, PkIndicesRef}; +use crate::executor::{Barrier, BoxedMessageStream, Executor, PkIndicesRef}; mod cache; mod sides; @@ -28,7 +28,7 @@ mod impl_; pub use impl_::LookupExecutorParams; -use super::ActorContextRef; +use super::{ActorContextRef, ExecutorInfo}; #[cfg(test)] mod tests; @@ -42,15 +42,11 @@ mod tests; pub struct LookupExecutor { ctx: ActorContextRef, + info: ExecutorInfo, + /// the data types of the produced data chunk inside lookup (before reordering) chunk_data_types: Vec, - /// The schema of the lookup executor (after reordering) - schema: Schema, - - /// The primary key indices of the schema (after reordering) - pk_indices: PkIndices, - /// The join side of the arrangement arrangement: ArrangeJoinSide, @@ -93,14 +89,14 @@ impl Executor for LookupExecutor { } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - "LookupExecutor" + &self.info.identity } } diff --git a/src/stream/src/executor/lookup/impl_.rs b/src/stream/src/executor/lookup/impl_.rs index e7f39c0247bf9..cc3afd2b2ec53 100644 --- a/src/stream/src/executor/lookup/impl_.rs +++ b/src/stream/src/executor/lookup/impl_.rs @@ -36,12 +36,13 @@ use crate::executor::error::{StreamExecutorError, StreamExecutorResult}; use crate::executor::lookup::cache::LookupCache; use crate::executor::lookup::sides::{ArrangeJoinSide, ArrangeMessage, StreamJoinSide}; use crate::executor::lookup::LookupExecutor; -use crate::executor::{ActorContextRef, Barrier, Executor, Message, PkIndices}; +use crate::executor::{ActorContextRef, Barrier, Executor, ExecutorInfo, Message}; use crate::task::AtomicU64Ref; /// Parameters for [`LookupExecutor`]. pub struct LookupExecutorParams { pub ctx: ActorContextRef, + pub info: ExecutorInfo, /// The side for arrangement. Currently, it should be a /// `MaterializeExecutor`. @@ -72,24 +73,6 @@ pub struct LookupExecutorParams { /// of this `arrangement_order_rules`. pub arrangement_order_rules: Vec, - /// Primary key indices of the lookup result (after reordering). - /// - /// [`LookupExecutor`] will lookup a row from the stream using the join key in the arrangement. - /// Therefore, the output of the [`LookupExecutor`] will be: - /// - /// ```plain - /// | stream columns | arrangement columns | - /// ``` - /// - /// ... and will be reordered by `output_column_reorder_idx`. - /// - /// The optimizer should select pk with pk of the stream columns, and pk of the original - /// materialized view (upstream of arrangement). - pub pk_indices: PkIndices, - - /// Schema of the lookup result (after reordering). - pub schema: Schema, - /// By default, the output of [`LookupExecutor`] is `stream columns + arrangement columns`. /// The executor will do a reorder of columns before producing output, so that data can be /// consistent among A lookup B and B lookup A. @@ -122,15 +105,14 @@ impl LookupExecutor { pub fn new(params: LookupExecutorParams) -> Self { let LookupExecutorParams { ctx, + info, arrangement, stream, arrangement_col_descs, arrangement_order_rules, - pk_indices, use_current_epoch, stream_join_key_indices, arrange_join_key_indices, - schema: output_schema, column_mapping, storage_table, watermark_epoch, @@ -195,7 +177,7 @@ impl LookupExecutor { // check the inferred schema is really the same as the output schema of the lookup executor. assert_eq!( - output_schema + info.schema .fields .iter() .map(|x| x.data_type()) @@ -216,9 +198,8 @@ impl LookupExecutor { Self { ctx, + info, chunk_data_types, - schema: output_schema, - pk_indices, last_barrier: None, stream_executor: Some(stream), arrangement_executor: Some(arrangement), diff --git a/src/stream/src/executor/lookup/tests.rs b/src/stream/src/executor/lookup/tests.rs index 5fda9504d17f7..a87d42b142ddc 100644 --- a/src/stream/src/executor/lookup/tests.rs +++ b/src/stream/src/executor/lookup/tests.rs @@ -30,7 +30,8 @@ use crate::executor::lookup::impl_::LookupExecutorParams; use crate::executor::lookup::LookupExecutor; use crate::executor::test_utils::*; use crate::executor::{ - ActorContext, Barrier, BoxedMessageStream, Executor, MaterializeExecutor, Message, PkIndices, + ActorContext, Barrier, BoxedMessageStream, Executor, ExecutorInfo, MaterializeExecutor, + Message, PkIndices, }; fn arrangement_col_descs() -> Vec { @@ -193,23 +194,27 @@ async fn test_lookup_this_epoch() { let table_id = TableId::new(1); let arrangement = create_arrangement(table_id, store.clone()).await; let stream = create_source(); + let info = ExecutorInfo { + schema: Schema::new(vec![ + Field::with_name(DataType::Int64, "join_column"), + Field::with_name(DataType::Int64, "rowid_column"), + Field::with_name(DataType::Int64, "rowid_column"), + Field::with_name(DataType::Int64, "join_column"), + ]), + pk_indices: vec![1, 2], + identity: "LookupExecutor".to_string(), + }; let lookup_executor = Box::new(LookupExecutor::new(LookupExecutorParams { ctx: ActorContext::create(0), + info, arrangement, stream, arrangement_col_descs: arrangement_col_descs(), arrangement_order_rules: arrangement_col_arrange_rules_join_key(), - pk_indices: vec![1, 2], use_current_epoch: true, stream_join_key_indices: vec![0], arrange_join_key_indices: vec![1], column_mapping: vec![2, 3, 0, 1], - schema: Schema::new(vec![ - Field::with_name(DataType::Int64, "join_column"), - Field::with_name(DataType::Int64, "rowid_column"), - Field::with_name(DataType::Int64, "rowid_column"), - Field::with_name(DataType::Int64, "join_column"), - ]), storage_table: StorageTable::for_test( store.clone(), table_id, @@ -263,23 +268,27 @@ async fn test_lookup_last_epoch() { let table_id = TableId::new(1); let arrangement = create_arrangement(table_id, store.clone()).await; let stream = create_source(); + let info = ExecutorInfo { + schema: Schema::new(vec![ + Field::with_name(DataType::Int64, "rowid_column"), + Field::with_name(DataType::Int64, "join_column"), + Field::with_name(DataType::Int64, "join_column"), + Field::with_name(DataType::Int64, "rowid_column"), + ]), + pk_indices: vec![1, 2], + identity: "LookupExecutor".to_string(), + }; let lookup_executor = Box::new(LookupExecutor::new(LookupExecutorParams { ctx: ActorContext::create(0), + info, arrangement, stream, arrangement_col_descs: arrangement_col_descs(), arrangement_order_rules: arrangement_col_arrange_rules_join_key(), - pk_indices: vec![1, 2], use_current_epoch: false, stream_join_key_indices: vec![0], arrange_join_key_indices: vec![1], column_mapping: vec![0, 1, 2, 3], - schema: Schema::new(vec![ - Field::with_name(DataType::Int64, "rowid_column"), - Field::with_name(DataType::Int64, "join_column"), - Field::with_name(DataType::Int64, "join_column"), - Field::with_name(DataType::Int64, "rowid_column"), - ]), storage_table: StorageTable::for_test( store.clone(), table_id, diff --git a/src/stream/src/executor/lookup_union.rs b/src/stream/src/executor/lookup_union.rs index 5460a57d3cb9e..422a4f2ba6072 100644 --- a/src/stream/src/executor/lookup_union.rs +++ b/src/stream/src/executor/lookup_union.rs @@ -29,8 +29,8 @@ use crate::executor::{BoxedMessageStream, ExecutorInfo}; /// pipe the second, and finally the first. In the future we could have more efficient /// implementation. pub struct LookupUnionExecutor { - inputs: Vec, info: ExecutorInfo, + inputs: Vec, order: Vec, } diff --git a/src/stream/src/from_proto/dynamic_filter.rs b/src/stream/src/from_proto/dynamic_filter.rs index 81c268530789a..6fddd10ddd9c6 100644 --- a/src/stream/src/from_proto/dynamic_filter.rs +++ b/src/stream/src/from_proto/dynamic_filter.rs @@ -69,11 +69,10 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { Ok(Box::new(DynamicFilterExecutor::new( params.actor_context, + params.info, source_l, source_r, key_l, - params.pk_indices, - params.executor_id, comparator, state_table_l, state_table_r, @@ -86,11 +85,10 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { Ok(Box::new(DynamicFilterExecutor::new( params.actor_context, + params.info, source_l, source_r, key_l, - params.pk_indices, - params.executor_id, comparator, state_table_l, state_table_r, diff --git a/src/stream/src/from_proto/hash_join.rs b/src/stream/src/from_proto/hash_join.rs index 19b2760f2d91e..66612cc954e07 100644 --- a/src/stream/src/from_proto/hash_join.rs +++ b/src/stream/src/from_proto/hash_join.rs @@ -28,7 +28,7 @@ use super::*; use crate::common::table::state_table::StateTable; use crate::executor::hash_join::*; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ActorContextRef, PkIndices}; +use crate::executor::ActorContextRef; use crate::task::AtomicU64Ref; pub struct HashJoinExecutorBuilder; @@ -138,17 +138,15 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { let args = HashJoinExecutorDispatcherArgs { ctx: params.actor_context, + info: params.info, source_l, source_r, params_l, params_r, null_safe, - pk_indices: params.pk_indices, output_indices, - executor_id: params.executor_id, cond: condition, inequality_pairs, - op_info: params.op_info, state_table_l, degree_state_table_l, state_table_r, @@ -167,17 +165,15 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { struct HashJoinExecutorDispatcherArgs { ctx: ActorContextRef, + info: ExecutorInfo, source_l: Box, source_r: Box, params_l: JoinParams, params_r: JoinParams, null_safe: Vec, - pk_indices: PkIndices, output_indices: Vec, - executor_id: u64, cond: Option, inequality_pairs: Vec<(usize, usize, bool, Option)>, - op_info: String, state_table_l: StateTable, degree_state_table_l: StateTable, state_table_r: StateTable, @@ -200,17 +196,15 @@ impl HashKeyDispatcher for HashJoinExecutorDispatcherArgs { Ok(Box::new( HashJoinExecutor::::new( self.ctx, + self.info, self.source_l, self.source_r, self.params_l, self.params_r, self.null_safe, - self.pk_indices, self.output_indices, - self.executor_id, self.cond, self.inequality_pairs, - self.op_info, self.state_table_l, self.degree_state_table_l, self.state_table_r, diff --git a/src/stream/src/from_proto/lookup.rs b/src/stream/src/from_proto/lookup.rs index edf67e57303d7..960a3c211c320 100644 --- a/src/stream/src/from_proto/lookup.rs +++ b/src/stream/src/from_proto/lookup.rs @@ -125,12 +125,11 @@ impl ExecutorBuilder for LookupExecutorBuilder { Ok(Box::new(LookupExecutor::new(LookupExecutorParams { ctx: params.actor_context, - schema: params.schema, + info: params.info, arrangement, stream, arrangement_col_descs, arrangement_order_rules, - pk_indices: params.pk_indices, use_current_epoch: lookup.use_current_epoch, stream_join_key_indices: lookup.stream_key.iter().map(|x| *x as usize).collect(), arrange_join_key_indices: lookup.arrange_key.iter().map(|x| *x as usize).collect(), From 234b99b61734c9193234aebfbf58659775d52873 Mon Sep 17 00:00:00 2001 From: xfz <73645462+xuefengze@users.noreply.github.com> Date: Tue, 14 Nov 2023 14:04:31 +0800 Subject: [PATCH 64/77] fix: wrong ts_ms field in debezium json (#13391) Co-authored-by: Bohan Zhang --- e2e_test/sink/kafka/debezium1.result | 20 ++++++++-------- e2e_test/sink/kafka/debezium2.result | 22 ++++++++--------- e2e_test/sink/kafka/debezium3.result | 24 +++++++++---------- .../src/sink/formatter/debezium_json.rs | 4 ++-- 4 files changed, 35 insertions(+), 35 deletions(-) diff --git a/e2e_test/sink/kafka/debezium1.result b/e2e_test/sink/kafka/debezium1.result index bc92096cd2c62..e624706e0cd9d 100644 --- a/e2e_test/sink/kafka/debezium1.result +++ b/e2e_test/sink/kafka/debezium1.result @@ -1,10 +1,10 @@ -{"payload":{"id":10},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":10,"v_bigint":20674,"v_double":9042.404483827513,"v_float":19387.23828125,"v_integer":20674,"v_smallint":26951,"v_timestamp":1681404058888,"v_varchar":"0oVqRIHqkb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":1,"v_bigint":1872,"v_double":23956.39329760601,"v_float":26261.416015625,"v_integer":1872,"v_smallint":31031,"v_timestamp":1681453634104,"v_varchar":"8DfUFencLe"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":2},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":2,"v_bigint":4598,"v_double":31923.077305746086,"v_float":27031.224609375,"v_integer":4598,"v_smallint":22690,"v_timestamp":1681429444869,"v_varchar":"sIo1XXVeHZ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":3},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":3,"v_bigint":14894,"v_double":9742.475509566086,"v_float":2660.290283203125,"v_integer":5894,"v_smallint":5985,"v_timestamp":1681429011269,"v_varchar":"LVLAhd1pQv"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":4},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":4,"v_bigint":24962,"v_double":3119.719721891862,"v_float":21217.77734375,"v_integer":7406,"v_smallint":6306,"v_timestamp":1681434727993,"v_varchar":"ORjwy3oMNb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":5},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":5,"v_bigint":9253,"v_double":17464.91553421121,"v_float":22749.5,"v_integer":9253,"v_smallint":22765,"v_timestamp":1681444642324,"v_varchar":"sSkKswxrYd"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":6},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":6,"v_bigint":28842,"v_double":11210.458724794062,"v_float":5885.3681640625,"v_integer":10844,"v_smallint":4014,"v_timestamp":1681382522137,"v_varchar":"V4y71v4Gip"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":7},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":7,"v_bigint":15914,"v_double":10967.182297153104,"v_float":3946.743408203125,"v_integer":12652,"v_smallint":10324,"v_timestamp":1681447263083,"v_varchar":"YIVLnWxHyf"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":8},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":8,"v_bigint":28641,"v_double":993.408963466774,"v_float":13652.0732421875,"v_integer":19036,"v_smallint":194,"v_timestamp":1681393929356,"v_varchar":"lv7Eq3g8hx"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":9},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":9,"v_bigint":24837,"v_double":11615.276406159757,"v_float":20699.55859375,"v_integer":20090,"v_smallint":10028,"v_timestamp":1681389642487,"v_varchar":"nwRq4zejSQ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":10},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":10,"v_bigint":20674,"v_double":9042.404483827513,"v_float":19387.23828125,"v_integer":20674,"v_smallint":26951,"v_timestamp":1681404058888,"v_varchar":"0oVqRIHqkb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866731158},"ts_ms":1699866731158},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":1,"v_bigint":1872,"v_double":23956.39329760601,"v_float":26261.416015625,"v_integer":1872,"v_smallint":31031,"v_timestamp":1681453634104,"v_varchar":"8DfUFencLe"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866731158},"ts_ms":1699866731158},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":2},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":2,"v_bigint":4598,"v_double":31923.077305746086,"v_float":27031.224609375,"v_integer":4598,"v_smallint":22690,"v_timestamp":1681429444869,"v_varchar":"sIo1XXVeHZ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866731158},"ts_ms":1699866731158},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":3},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":3,"v_bigint":14894,"v_double":9742.475509566086,"v_float":2660.290283203125,"v_integer":5894,"v_smallint":5985,"v_timestamp":1681429011269,"v_varchar":"LVLAhd1pQv"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866731158},"ts_ms":1699866731158},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":4},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":4,"v_bigint":24962,"v_double":3119.719721891862,"v_float":21217.77734375,"v_integer":7406,"v_smallint":6306,"v_timestamp":1681434727993,"v_varchar":"ORjwy3oMNb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866731158},"ts_ms":1699866731158},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":5},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":5,"v_bigint":9253,"v_double":17464.91553421121,"v_float":22749.5,"v_integer":9253,"v_smallint":22765,"v_timestamp":1681444642324,"v_varchar":"sSkKswxrYd"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866731158},"ts_ms":1699866731158},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":6},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":6,"v_bigint":28842,"v_double":11210.458724794062,"v_float":5885.3681640625,"v_integer":10844,"v_smallint":4014,"v_timestamp":1681382522137,"v_varchar":"V4y71v4Gip"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866731158},"ts_ms":1699866731158},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":7},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":7,"v_bigint":15914,"v_double":10967.182297153104,"v_float":3946.743408203125,"v_integer":12652,"v_smallint":10324,"v_timestamp":1681447263083,"v_varchar":"YIVLnWxHyf"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866731158},"ts_ms":1699866731158},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":8},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":8,"v_bigint":28641,"v_double":993.408963466774,"v_float":13652.0732421875,"v_integer":19036,"v_smallint":194,"v_timestamp":1681393929356,"v_varchar":"lv7Eq3g8hx"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866731158},"ts_ms":1699866731158},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":9},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":9,"v_bigint":24837,"v_double":11615.276406159757,"v_float":20699.55859375,"v_integer":20090,"v_smallint":10028,"v_timestamp":1681389642487,"v_varchar":"nwRq4zejSQ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866731158},"ts_ms":1699866731158},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} diff --git a/e2e_test/sink/kafka/debezium2.result b/e2e_test/sink/kafka/debezium2.result index c1f1524c9a825..86b569f9325b8 100644 --- a/e2e_test/sink/kafka/debezium2.result +++ b/e2e_test/sink/kafka/debezium2.result @@ -1,11 +1,11 @@ -{"payload":{"id":10},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":10,"v_bigint":20674,"v_double":9042.404483827513,"v_float":19387.23828125,"v_integer":20674,"v_smallint":26951,"v_timestamp":1681404058888,"v_varchar":"0oVqRIHqkb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":1,"v_bigint":0,"v_double":0.0,"v_float":0.0,"v_integer":0,"v_smallint":0,"v_timestamp":0,"v_varchar":""},"before":{"id":1,"v_bigint":1872,"v_double":23956.39329760601,"v_float":26261.416015625,"v_integer":1872,"v_smallint":31031,"v_timestamp":1681453634104,"v_varchar":"8DfUFencLe"},"op":"u","source":{"db":"dev","table":"t_kafka","ts_ms":1696838696640},"ts_ms":1696838696640},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":1,"v_bigint":1872,"v_double":23956.39329760601,"v_float":26261.416015625,"v_integer":1872,"v_smallint":31031,"v_timestamp":1681453634104,"v_varchar":"8DfUFencLe"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":2},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":2,"v_bigint":4598,"v_double":31923.077305746086,"v_float":27031.224609375,"v_integer":4598,"v_smallint":22690,"v_timestamp":1681429444869,"v_varchar":"sIo1XXVeHZ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":3},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":3,"v_bigint":14894,"v_double":9742.475509566086,"v_float":2660.290283203125,"v_integer":5894,"v_smallint":5985,"v_timestamp":1681429011269,"v_varchar":"LVLAhd1pQv"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":4},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":4,"v_bigint":24962,"v_double":3119.719721891862,"v_float":21217.77734375,"v_integer":7406,"v_smallint":6306,"v_timestamp":1681434727993,"v_varchar":"ORjwy3oMNb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":5},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":5,"v_bigint":9253,"v_double":17464.91553421121,"v_float":22749.5,"v_integer":9253,"v_smallint":22765,"v_timestamp":1681444642324,"v_varchar":"sSkKswxrYd"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":6},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":6,"v_bigint":28842,"v_double":11210.458724794062,"v_float":5885.3681640625,"v_integer":10844,"v_smallint":4014,"v_timestamp":1681382522137,"v_varchar":"V4y71v4Gip"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":7},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":7,"v_bigint":15914,"v_double":10967.182297153104,"v_float":3946.743408203125,"v_integer":12652,"v_smallint":10324,"v_timestamp":1681447263083,"v_varchar":"YIVLnWxHyf"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":8},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":8,"v_bigint":28641,"v_double":993.408963466774,"v_float":13652.0732421875,"v_integer":19036,"v_smallint":194,"v_timestamp":1681393929356,"v_varchar":"lv7Eq3g8hx"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":9},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":9,"v_bigint":24837,"v_double":11615.276406159757,"v_float":20699.55859375,"v_integer":20090,"v_smallint":10028,"v_timestamp":1681389642487,"v_varchar":"nwRq4zejSQ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":10},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":10,"v_bigint":20674,"v_double":9042.404483827513,"v_float":19387.23828125,"v_integer":20674,"v_smallint":26951,"v_timestamp":1681404058888,"v_varchar":"0oVqRIHqkb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866896865},"ts_ms":1699866896865},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":1,"v_bigint":0,"v_double":0.0,"v_float":0.0,"v_integer":0,"v_smallint":0,"v_timestamp":0,"v_varchar":""},"before":{"id":1,"v_bigint":1872,"v_double":23956.39329760601,"v_float":26261.416015625,"v_integer":1872,"v_smallint":31031,"v_timestamp":1681453634104,"v_varchar":"8DfUFencLe"},"op":"u","source":{"db":"dev","table":"t_kafka","ts_ms":1699866905338},"ts_ms":1699866905338},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":1,"v_bigint":1872,"v_double":23956.39329760601,"v_float":26261.416015625,"v_integer":1872,"v_smallint":31031,"v_timestamp":1681453634104,"v_varchar":"8DfUFencLe"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866896865},"ts_ms":1699866896865},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":2},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":2,"v_bigint":4598,"v_double":31923.077305746086,"v_float":27031.224609375,"v_integer":4598,"v_smallint":22690,"v_timestamp":1681429444869,"v_varchar":"sIo1XXVeHZ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866896865},"ts_ms":1699866896865},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":3},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":3,"v_bigint":14894,"v_double":9742.475509566086,"v_float":2660.290283203125,"v_integer":5894,"v_smallint":5985,"v_timestamp":1681429011269,"v_varchar":"LVLAhd1pQv"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866896865},"ts_ms":1699866896865},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":4},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":4,"v_bigint":24962,"v_double":3119.719721891862,"v_float":21217.77734375,"v_integer":7406,"v_smallint":6306,"v_timestamp":1681434727993,"v_varchar":"ORjwy3oMNb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866896865},"ts_ms":1699866896865},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":5},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":5,"v_bigint":9253,"v_double":17464.91553421121,"v_float":22749.5,"v_integer":9253,"v_smallint":22765,"v_timestamp":1681444642324,"v_varchar":"sSkKswxrYd"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866896865},"ts_ms":1699866896865},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":6},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":6,"v_bigint":28842,"v_double":11210.458724794062,"v_float":5885.3681640625,"v_integer":10844,"v_smallint":4014,"v_timestamp":1681382522137,"v_varchar":"V4y71v4Gip"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866896865},"ts_ms":1699866896865},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":7},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":7,"v_bigint":15914,"v_double":10967.182297153104,"v_float":3946.743408203125,"v_integer":12652,"v_smallint":10324,"v_timestamp":1681447263083,"v_varchar":"YIVLnWxHyf"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866896865},"ts_ms":1699866896865},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":8},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":8,"v_bigint":28641,"v_double":993.408963466774,"v_float":13652.0732421875,"v_integer":19036,"v_smallint":194,"v_timestamp":1681393929356,"v_varchar":"lv7Eq3g8hx"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866896865},"ts_ms":1699866896865},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":9},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":9,"v_bigint":24837,"v_double":11615.276406159757,"v_float":20699.55859375,"v_integer":20090,"v_smallint":10028,"v_timestamp":1681389642487,"v_varchar":"nwRq4zejSQ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699866896865},"ts_ms":1699866896865},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} diff --git a/e2e_test/sink/kafka/debezium3.result b/e2e_test/sink/kafka/debezium3.result index 2445a019d7d44..bafc02f54d22b 100644 --- a/e2e_test/sink/kafka/debezium3.result +++ b/e2e_test/sink/kafka/debezium3.result @@ -1,13 +1,13 @@ -{"payload":{"id":10},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":10,"v_bigint":20674,"v_double":9042.404483827513,"v_float":19387.23828125,"v_integer":20674,"v_smallint":26951,"v_timestamp":1681404058888,"v_varchar":"0oVqRIHqkb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":10},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":10,"v_bigint":20674,"v_double":9042.404483827513,"v_float":19387.23828125,"v_integer":20674,"v_smallint":26951,"v_timestamp":1681404058888,"v_varchar":"0oVqRIHqkb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699867240589},"ts_ms":1699867240589},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} {"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} null -{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":null,"before":{"id":1,"v_bigint":0,"v_double":0.0,"v_float":0.0,"v_integer":0,"v_smallint":0,"v_timestamp":0,"v_varchar":""},"op":"d","source":{"db":"dev","table":"t_kafka","ts_ms":1696838869564},"ts_ms":1696838869564},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":1,"v_bigint":0,"v_double":0.0,"v_float":0.0,"v_integer":0,"v_smallint":0,"v_timestamp":0,"v_varchar":""},"before":{"id":1,"v_bigint":1872,"v_double":23956.39329760601,"v_float":26261.416015625,"v_integer":1872,"v_smallint":31031,"v_timestamp":1681453634104,"v_varchar":"8DfUFencLe"},"op":"u","source":{"db":"dev","table":"t_kafka","ts_ms":1696838696640},"ts_ms":1696838696640},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":1,"v_bigint":1872,"v_double":23956.39329760601,"v_float":26261.416015625,"v_integer":1872,"v_smallint":31031,"v_timestamp":1681453634104,"v_varchar":"8DfUFencLe"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":2},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":2,"v_bigint":4598,"v_double":31923.077305746086,"v_float":27031.224609375,"v_integer":4598,"v_smallint":22690,"v_timestamp":1681429444869,"v_varchar":"sIo1XXVeHZ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":3},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":3,"v_bigint":14894,"v_double":9742.475509566086,"v_float":2660.290283203125,"v_integer":5894,"v_smallint":5985,"v_timestamp":1681429011269,"v_varchar":"LVLAhd1pQv"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":4},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":4,"v_bigint":24962,"v_double":3119.719721891862,"v_float":21217.77734375,"v_integer":7406,"v_smallint":6306,"v_timestamp":1681434727993,"v_varchar":"ORjwy3oMNb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":5},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":5,"v_bigint":9253,"v_double":17464.91553421121,"v_float":22749.5,"v_integer":9253,"v_smallint":22765,"v_timestamp":1681444642324,"v_varchar":"sSkKswxrYd"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":6},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":6,"v_bigint":28842,"v_double":11210.458724794062,"v_float":5885.3681640625,"v_integer":10844,"v_smallint":4014,"v_timestamp":1681382522137,"v_varchar":"V4y71v4Gip"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":7},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":7,"v_bigint":15914,"v_double":10967.182297153104,"v_float":3946.743408203125,"v_integer":12652,"v_smallint":10324,"v_timestamp":1681447263083,"v_varchar":"YIVLnWxHyf"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":8},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":8,"v_bigint":28641,"v_double":993.408963466774,"v_float":13652.0732421875,"v_integer":19036,"v_smallint":194,"v_timestamp":1681393929356,"v_varchar":"lv7Eq3g8hx"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} -{"payload":{"id":9},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":9,"v_bigint":24837,"v_double":11615.276406159757,"v_float":20699.55859375,"v_integer":20090,"v_smallint":10028,"v_timestamp":1681389642487,"v_varchar":"nwRq4zejSQ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1696838120862},"ts_ms":1696838120862},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":null,"before":{"id":1,"v_bigint":0,"v_double":0.0,"v_float":0.0,"v_integer":0,"v_smallint":0,"v_timestamp":0,"v_varchar":""},"op":"d","source":{"db":"dev","table":"t_kafka","ts_ms":1699867254063},"ts_ms":1699867254063},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":1,"v_bigint":0,"v_double":0.0,"v_float":0.0,"v_integer":0,"v_smallint":0,"v_timestamp":0,"v_varchar":""},"before":{"id":1,"v_bigint":1872,"v_double":23956.39329760601,"v_float":26261.416015625,"v_integer":1872,"v_smallint":31031,"v_timestamp":1681453634104,"v_varchar":"8DfUFencLe"},"op":"u","source":{"db":"dev","table":"t_kafka","ts_ms":1699867248763},"ts_ms":1699867248763},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":1},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":1,"v_bigint":1872,"v_double":23956.39329760601,"v_float":26261.416015625,"v_integer":1872,"v_smallint":31031,"v_timestamp":1681453634104,"v_varchar":"8DfUFencLe"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699867240589},"ts_ms":1699867240589},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":2},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":2,"v_bigint":4598,"v_double":31923.077305746086,"v_float":27031.224609375,"v_integer":4598,"v_smallint":22690,"v_timestamp":1681429444869,"v_varchar":"sIo1XXVeHZ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699867240590},"ts_ms":1699867240590},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":3},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":3,"v_bigint":14894,"v_double":9742.475509566086,"v_float":2660.290283203125,"v_integer":5894,"v_smallint":5985,"v_timestamp":1681429011269,"v_varchar":"LVLAhd1pQv"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699867240589},"ts_ms":1699867240589},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":4},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":4,"v_bigint":24962,"v_double":3119.719721891862,"v_float":21217.77734375,"v_integer":7406,"v_smallint":6306,"v_timestamp":1681434727993,"v_varchar":"ORjwy3oMNb"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699867240589},"ts_ms":1699867240589},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":5},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":5,"v_bigint":9253,"v_double":17464.91553421121,"v_float":22749.5,"v_integer":9253,"v_smallint":22765,"v_timestamp":1681444642324,"v_varchar":"sSkKswxrYd"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699867240590},"ts_ms":1699867240590},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":6},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":6,"v_bigint":28842,"v_double":11210.458724794062,"v_float":5885.3681640625,"v_integer":10844,"v_smallint":4014,"v_timestamp":1681382522137,"v_varchar":"V4y71v4Gip"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699867240589},"ts_ms":1699867240589},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":7},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":7,"v_bigint":15914,"v_double":10967.182297153104,"v_float":3946.743408203125,"v_integer":12652,"v_smallint":10324,"v_timestamp":1681447263083,"v_varchar":"YIVLnWxHyf"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699867240590},"ts_ms":1699867240590},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":8},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":8,"v_bigint":28641,"v_double":993.408963466774,"v_float":13652.0732421875,"v_integer":19036,"v_smallint":194,"v_timestamp":1681393929356,"v_varchar":"lv7Eq3g8hx"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699867240589},"ts_ms":1699867240589},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} +{"payload":{"id":9},"schema":{"fields":[{"field":"id","optional":true,"type":"int32"}],"name":"RisingWave.dev.t_kafka.Key","optional":false,"type":"struct"}} {"payload":{"after":{"id":9,"v_bigint":24837,"v_double":11615.276406159757,"v_float":20699.55859375,"v_integer":20090,"v_smallint":10028,"v_timestamp":1681389642487,"v_varchar":"nwRq4zejSQ"},"before":null,"op":"c","source":{"db":"dev","table":"t_kafka","ts_ms":1699867240590},"ts_ms":1699867240590},"schema":{"fields":[{"field":"before","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"id","optional":true,"type":"int32"},{"field":"v_varchar","optional":true,"type":"string"},{"field":"v_smallint","optional":true,"type":"int16"},{"field":"v_integer","optional":true,"type":"int32"},{"field":"v_bigint","optional":true,"type":"int64"},{"field":"v_float","optional":true,"type":"float"},{"field":"v_double","optional":true,"type":"double"},{"field":"v_timestamp","optional":true,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.dev.t_kafka.Envelope","optional":false,"type":"struct"}} diff --git a/src/connector/src/sink/formatter/debezium_json.rs b/src/connector/src/sink/formatter/debezium_json.rs index a40789f7d9c95..5948082c2edde 100644 --- a/src/connector/src/sink/formatter/debezium_json.rs +++ b/src/connector/src/sink/formatter/debezium_json.rs @@ -225,7 +225,7 @@ pub(crate) fn schema_to_json(schema: &Schema, db_name: &str, sink_from_name: &st json!({ "type": "int64", "optional": false, - "field": "table" + "field": "ts_ms" }), ], @@ -313,7 +313,7 @@ mod tests { use super::*; use crate::sink::utils::chunk_to_json; - const SCHEMA_JSON_RESULT: &str = r#"{"fields":[{"field":"before","fields":[{"field":"v1","optional":true,"type":"int32"},{"field":"v2","optional":true,"type":"float"},{"field":"v3","optional":true,"type":"string"}],"name":"RisingWave.test_db.test_table.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"v1","optional":true,"type":"int32"},{"field":"v2","optional":true,"type":"float"},{"field":"v3","optional":true,"type":"string"}],"name":"RisingWave.test_db.test_table.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"table","optional":false,"type":"int64"}],"name":"RisingWave.test_db.test_table.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.test_db.test_table.Envelope","optional":false,"type":"struct"}"#; + const SCHEMA_JSON_RESULT: &str = r#"{"fields":[{"field":"before","fields":[{"field":"v1","optional":true,"type":"int32"},{"field":"v2","optional":true,"type":"float"},{"field":"v3","optional":true,"type":"string"}],"name":"RisingWave.test_db.test_table.Key","optional":true,"type":"struct"},{"field":"after","fields":[{"field":"v1","optional":true,"type":"int32"},{"field":"v2","optional":true,"type":"float"},{"field":"v3","optional":true,"type":"string"}],"name":"RisingWave.test_db.test_table.Key","optional":true,"type":"struct"},{"field":"source","fields":[{"field":"db","optional":false,"type":"string"},{"field":"table","optional":true,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.test_db.test_table.Source","optional":false,"type":"struct"},{"field":"op","optional":false,"type":"string"},{"field":"ts_ms","optional":false,"type":"int64"}],"name":"RisingWave.test_db.test_table.Envelope","optional":false,"type":"struct"}"#; #[test] fn test_chunk_to_json() -> Result<()> { From 09d312c108394fa48b6747e62a03c3f77d3d4ed2 Mon Sep 17 00:00:00 2001 From: Croxx Date: Tue, 14 Nov 2023 14:29:54 +0800 Subject: [PATCH 65/77] refactor: bump aws sdk, use rustls in s3 client (#13286) Signed-off-by: MrCroxx --- Cargo.lock | 636 +++++++++++------- Cargo.toml | 40 +- src/bench/Cargo.toml | 1 + src/bench/s3_bench/main.rs | 2 +- src/connector/Cargo.toml | 2 + src/connector/src/aws_auth.rs | 3 +- .../src/source/filesystem/s3/source/reader.rs | 17 +- .../src/source/kinesis/enumerator/client.rs | 2 +- .../src/source/kinesis/source/message.rs | 6 +- .../src/source/kinesis/source/reader.rs | 2 +- src/meta/src/rpc/cloud_provider.rs | 26 +- src/object_store/Cargo.toml | 7 +- src/object_store/src/object/error.rs | 16 +- src/object_store/src/object/s3.rs | 135 ++-- src/tests/simulation/Cargo.toml | 2 +- src/workspace-hack/Cargo.toml | 11 +- 16 files changed, 527 insertions(+), 381 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7091b2682c594..2ddc20b82e213 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -511,11 +511,11 @@ dependencies = [ "once_cell", "rand", "regex", - "ring", - "rustls 0.21.7", + "ring 0.16.20", + "rustls", "rustls-native-certs", "rustls-pemfile", - "rustls-webpki 0.101.4", + "rustls-webpki 0.101.7", "serde", "serde_json", "serde_nanos", @@ -524,7 +524,7 @@ dependencies = [ "time", "tokio", "tokio-retry", - "tokio-rustls 0.24.1", + "tokio-rustls", "tracing", "url", ] @@ -685,148 +685,144 @@ dependencies = [ [[package]] name = "aws-config" -version = "0.55.3" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcdcf0d683fe9c23d32cf5b53c9918ea0a500375a9fb20109802552658e576c9" +checksum = "7489a72550db3712fe3a0a92068f832d6270ff82f518b84a800af131f99570d7" dependencies = [ "aws-credential-types", "aws-http", + "aws-runtime", "aws-sdk-sts", "aws-smithy-async", - "aws-smithy-client", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", "bytes", - "fastrand 1.9.0", + "fastrand 2.0.0", "http", "hyper", "time", "tokio", - "tower", "tracing", ] [[package]] name = "aws-credential-types" -version = "0.55.3" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fcdb2f7acbc076ff5ad05e7864bdb191ca70a6fd07668dc3a1a8bcd051de5ae" +checksum = "80009846d61a0a4f9070d789cf0e64db284cba6984fae3871050d044e6569cd2" dependencies = [ "aws-smithy-async", + "aws-smithy-runtime-api", "aws-smithy-types", - "fastrand 1.9.0", - "tokio", - "tracing", "zeroize", ] [[package]] -name = "aws-endpoint" -version = "0.55.3" +name = "aws-http" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cce1c41a6cfaa726adee9ebb9a56fcd2bbfd8be49fd8a04c5e20fd968330b04" +checksum = "1e65730b741a5f6422fd338bf6f76b7956b090affeaa045e78fca8c4186e0fd5" dependencies = [ "aws-smithy-http", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", + "bytes", "http", - "regex", + "http-body", + "pin-project-lite", "tracing", ] [[package]] -name = "aws-http" -version = "0.55.3" +name = "aws-runtime" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aadbc44e7a8f3e71c8b374e03ecd972869eb91dd2bc89ed018954a52ba84bc44" +checksum = "1d2414b96071ae840b97c0cc1d44b248d5607d648593cdf474f3fb5465572898" dependencies = [ "aws-credential-types", + "aws-http", + "aws-sigv4", + "aws-smithy-async", + "aws-smithy-eventstream", "aws-smithy-http", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", - "bytes", + "fastrand 2.0.0", "http", - "http-body", - "lazy_static", "percent-encoding", - "pin-project-lite", "tracing", + "uuid", ] [[package]] name = "aws-sdk-ec2" -version = "0.28.0" +version = "0.35.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eab2493c5857725eeafe12ec66ba4ce6feb3355e3af6828d9ef28d6152972a27" +checksum = "7d3492d8b17bc47532abee2d83a3f861997a7017ca548abb0ab4a26d5e6a691b" dependencies = [ "aws-credential-types", - "aws-endpoint", "aws-http", - "aws-sig-auth", + "aws-runtime", "aws-smithy-async", - "aws-smithy-client", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", "aws-smithy-query", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-smithy-xml", "aws-types", - "bytes", - "fastrand 1.9.0", + "fastrand 2.0.0", "http", "regex", - "tokio-stream", - "tower", "tracing", ] [[package]] name = "aws-sdk-kinesis" -version = "0.28.0" +version = "0.35.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca88060b315da80279486d079a2c5c27891fc60a7e770526e50ad5d98551f650" +checksum = "5acef0ae3beb145ba41e640fe33b2b5ac021218ee0b42199779393d2b7e4e2cc" dependencies = [ "aws-credential-types", - "aws-endpoint", "aws-http", - "aws-sig-auth", + "aws-runtime", "aws-smithy-async", - "aws-smithy-client", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", "bytes", "http", "regex", - "tokio-stream", - "tower", "tracing", ] [[package]] name = "aws-sdk-s3" -version = "0.28.0" +version = "0.35.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fba197193cbb4bcb6aad8d99796b2291f36fa89562ded5d4501363055b0de89f" +checksum = "84022763485483ea17d417f9832d5da198bc36829b59f086c0d35ecd2ce59991" dependencies = [ "aws-credential-types", - "aws-endpoint", "aws-http", - "aws-sig-auth", + "aws-runtime", "aws-sigv4", "aws-smithy-async", "aws-smithy-checksums", - "aws-smithy-client", "aws-smithy-eventstream", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-smithy-xml", "aws-types", @@ -836,91 +832,76 @@ dependencies = [ "once_cell", "percent-encoding", "regex", - "tokio-stream", - "tower", "tracing", "url", ] [[package]] name = "aws-sdk-sts" -version = "0.28.0" +version = "0.35.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "265fac131fbfc188e5c3d96652ea90ecc676a934e3174eaaee523c6cec040b3b" +checksum = "51b1a8ae5c7098502a3e6d4130dbee1e1d3fcb8dc5d65cecab39e01d595f90f6" dependencies = [ "aws-credential-types", - "aws-endpoint", "aws-http", - "aws-sig-auth", + "aws-runtime", "aws-smithy-async", - "aws-smithy-client", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", "aws-smithy-query", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-smithy-xml", "aws-types", - "bytes", "http", "regex", - "tower", - "tracing", -] - -[[package]] -name = "aws-sig-auth" -version = "0.55.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b94acb10af0c879ecd5c7bdf51cda6679a0a4f4643ce630905a77673bfa3c61" -dependencies = [ - "aws-credential-types", - "aws-sigv4", - "aws-smithy-eventstream", - "aws-smithy-http", - "aws-types", - "http", "tracing", ] [[package]] name = "aws-sigv4" -version = "0.55.3" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d2ce6f507be68e968a33485ced670111d1cbad161ddbbab1e313c03d37d8f4c" +checksum = "3347c738e0a8449020877d319cda56da74d6e8aba9fff210720fac66cae3c7f4" dependencies = [ + "aws-credential-types", "aws-smithy-eventstream", "aws-smithy-http", + "aws-smithy-runtime-api", "bytes", "form_urlencoded", "hex", "hmac", "http", + "num-bigint", "once_cell", + "p256 0.11.1", "percent-encoding", "regex", + "ring 0.17.5", "sha2", "time", "tracing", + "zeroize", ] [[package]] name = "aws-smithy-async" -version = "0.55.3" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13bda3996044c202d75b91afeb11a9afae9db9a721c6a7a427410018e286b880" +checksum = "b4b65a284265d3eec6cc9f1daef2d0cc3b78684b712cb6c7f1d0f665456b7604" dependencies = [ "futures-util", "pin-project-lite", "tokio", - "tokio-stream", ] [[package]] name = "aws-smithy-checksums" -version = "0.55.3" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07ed8b96d95402f3f6b8b57eb4e0e45ee365f78b1a924faf20ff6e97abf1eae6" +checksum = "d40f1d5a222ba11ac7d6b20f3668ae282970e50615fa5ee1dd8ac8180c0c1803" dependencies = [ "aws-smithy-http", "aws-smithy-types", @@ -937,36 +918,11 @@ dependencies = [ "tracing", ] -[[package]] -name = "aws-smithy-client" -version = "0.55.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a86aa6e21e86c4252ad6a0e3e74da9617295d8d6e374d552be7d3059c41cedd" -dependencies = [ - "aws-smithy-async", - "aws-smithy-http", - "aws-smithy-http-tower", - "aws-smithy-types", - "bytes", - "fastrand 1.9.0", - "http", - "http-body", - "hyper", - "hyper-rustls 0.23.2", - "hyper-tls", - "lazy_static", - "pin-project-lite", - "rustls 0.20.9", - "tokio", - "tower", - "tracing", -] - [[package]] name = "aws-smithy-eventstream" -version = "0.55.3" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "460c8da5110835e3d9a717c61f5556b20d03c32a1dec57f8fc559b360f733bb8" +checksum = "b16e7ecebc2b083a1b138868a46a343204a6097f343c4830a8b22b3a0d30013e" dependencies = [ "aws-smithy-types", "bytes", @@ -975,94 +931,125 @@ dependencies = [ [[package]] name = "aws-smithy-http" -version = "0.55.3" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b3b693869133551f135e1f2c77cb0b8277d9e3e17feaf2213f735857c4f0d28" +checksum = "715aeb61fb743848d5d398ce6fb1259f5eba5e13dceec5d5064cada1a181d38d" dependencies = [ "aws-smithy-eventstream", + "aws-smithy-runtime-api", "aws-smithy-types", "bytes", "bytes-utils", "futures-core", "http", "http-body", - "hyper", "once_cell", "percent-encoding", "pin-project-lite", "pin-utils", - "tokio", - "tokio-util", "tracing", ] [[package]] -name = "aws-smithy-http-tower" -version = "0.55.3" +name = "aws-smithy-json" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ae4f6c5798a247fac98a867698197d9ac22643596dc3777f0c76b91917616b9" +checksum = "de21d368dcd5cab17033406ea6e7351b091164b208381de837510bd7558c0f30" dependencies = [ - "aws-smithy-http", "aws-smithy-types", - "bytes", - "http", - "http-body", - "pin-project-lite", - "tower", - "tracing", ] [[package]] -name = "aws-smithy-json" -version = "0.55.3" +name = "aws-smithy-query" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23f9f42fbfa96d095194a632fbac19f60077748eba536eb0b9fecc28659807f8" +checksum = "9e5ace389c7e4def130bff7275647481c8d49b867909ca61d5dc9a809b3632f3" dependencies = [ "aws-smithy-types", + "urlencoding", ] [[package]] -name = "aws-smithy-query" -version = "0.55.3" +name = "aws-smithy-runtime" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98819eb0b04020a1c791903533b638534ae6c12e2aceda3e6e6fba015608d51d" +checksum = "fb4395310662d10f1847324af5fe43e621922cba03b1aa6d26c21096e18a4e79" dependencies = [ + "aws-smithy-async", + "aws-smithy-http", + "aws-smithy-runtime-api", "aws-smithy-types", - "urlencoding", + "bytes", + "fastrand 2.0.0", + "http", + "http-body", + "hyper", + "hyper-rustls", + "once_cell", + "pin-project-lite", + "pin-utils", + "rustls", + "tokio", + "tracing", +] + +[[package]] +name = "aws-smithy-runtime-api" +version = "0.57.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30e27594c06f5b36e97d18ef26ed693f1d4c7167b9bbb544b3a9bb653f9f7035" +dependencies = [ + "aws-smithy-async", + "aws-smithy-types", + "bytes", + "http", + "pin-project-lite", + "tokio", + "tracing", ] [[package]] name = "aws-smithy-types" -version = "0.55.3" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16a3d0bf4f324f4ef9793b86a1701d9700fbcdbd12a846da45eed104c634c6e8" +checksum = "5d36f1723ed61e82094498e7283510fe21484b73c215c33874c81a84411b5bdc" dependencies = [ "base64-simd", + "bytes", + "bytes-utils", + "futures-core", + "http", + "http-body", + "hyper", "itoa", "num-integer", + "pin-project-lite", + "pin-utils", "ryu", + "serde", "time", + "tokio", + "tokio-util", ] [[package]] name = "aws-smithy-xml" -version = "0.55.3" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1b9d12875731bd07e767be7baad95700c3137b56730ec9ddeedb52a5e5ca63b" +checksum = "68225c8d3e3e6c565a3cf764aa82440837ef15c33d1dd7205e15715444e4b4ad" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "0.55.3" +version = "0.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6dd209616cc8d7bfb82f87811a5c655dc97537f592689b18743bddf5dc5c4829" +checksum = "acdc27aac60f715bab25f5d758ba5651b80aae791c48e9871ffe298683f00a2b" dependencies = [ "aws-credential-types", "aws-smithy-async", - "aws-smithy-client", - "aws-smithy-http", + "aws-smithy-runtime-api", "aws-smithy-types", "http", "rustc_version", @@ -1158,6 +1145,12 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "base16ct" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "349a06037c7bf932dd7e7d1f653678b2038b9ad46a74102f1fc7bd7872678cce" + [[package]] name = "base16ct" version = "0.2.0" @@ -2194,6 +2187,18 @@ version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" +[[package]] +name = "crypto-bigint" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef2b4b23cddf68b89b8f8069890e8c270d54e2d5fe1b143820234805e4cb17ef" +dependencies = [ + "generic-array", + "rand_core", + "subtle", + "zeroize", +] + [[package]] name = "crypto-bigint" version = "0.5.3" @@ -2473,6 +2478,16 @@ dependencies = [ "uuid", ] +[[package]] +name = "der" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1a467a65c5e759bce6e65eaf91cc29f466cdc57cb65777bd646872a8a1fd4de" +dependencies = [ + "const-oid", + "zeroize", +] + [[package]] name = "der" version = "0.7.8" @@ -2655,18 +2670,30 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "49457524c7e65648794c98283282a0b7c73b10018e7091f1cdcfff314fd7ae59" +[[package]] +name = "ecdsa" +version = "0.14.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "413301934810f597c1d19ca71c8710e99a3f1ba28a0d2ebc01551a2daeea3c5c" +dependencies = [ + "der 0.6.1", + "elliptic-curve 0.12.3", + "rfc6979 0.3.1", + "signature 1.6.4", +] + [[package]] name = "ecdsa" version = "0.16.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a4b1e0c257a9e9f25f90ff76d7a68360ed497ee519c8e428d1825ef0000799d4" dependencies = [ - "der", + "der 0.7.8", "digest", - "elliptic-curve", - "rfc6979", - "signature", - "spki", + "elliptic-curve 0.13.6", + "rfc6979 0.4.0", + "signature 2.0.0", + "spki 0.7.2", ] [[package]] @@ -2675,8 +2702,8 @@ version = "2.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "60f6d271ca33075c88028be6f04d502853d63a5ece419d269c15315d4fc1cf1d" dependencies = [ - "pkcs8", - "signature", + "pkcs8 0.10.2", + "signature 2.0.0", ] [[package]] @@ -2689,7 +2716,7 @@ dependencies = [ "ed25519", "serde", "sha2", - "signature", + "signature 2.0.0", "zeroize", ] @@ -2714,23 +2741,43 @@ dependencies = [ "serde", ] +[[package]] +name = "elliptic-curve" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7bb888ab5300a19b8e5bceef25ac745ad065f3c9f7efc6de1b91958110891d3" +dependencies = [ + "base16ct 0.1.1", + "crypto-bigint 0.4.9", + "der 0.6.1", + "digest", + "ff 0.12.1", + "generic-array", + "group 0.12.1", + "pkcs8 0.9.0", + "rand_core", + "sec1 0.3.0", + "subtle", + "zeroize", +] + [[package]] name = "elliptic-curve" version = "0.13.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d97ca172ae9dc9f9b779a6e3a65d308f2af74e5b8c921299075bdb4a0370e914" dependencies = [ - "base16ct", - "crypto-bigint", + "base16ct 0.2.0", + "crypto-bigint 0.5.3", "digest", - "ff", + "ff 0.13.0", "generic-array", - "group", + "group 0.13.0", "hkdf", "pem-rfc7468", - "pkcs8", + "pkcs8 0.10.2", "rand_core", - "sec1", + "sec1 0.7.3", "subtle", "zeroize", ] @@ -2986,6 +3033,16 @@ version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6999dc1837253364c2ebb0704ba97994bd874e8f195d665c50b7548f6ea92764" +[[package]] +name = "ff" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d013fc25338cc558c5c2cfbad646908fb23591e2404481826742b651c9af7160" +dependencies = [ + "rand_core", + "subtle", +] + [[package]] name = "ff" version = "0.13.0" @@ -3486,7 +3543,7 @@ dependencies = [ "async-trait", "dyn-clone", "hyper", - "hyper-rustls 0.24.1", + "hyper-rustls", "log", "reqwest", "serde", @@ -3525,8 +3582,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.9" -source = "git+https://github.com/madsim-rs/getrandom.git?rev=8daf97e#8daf97e4142635fe28543b2db9022f5e2544bb5c" +version = "0.2.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" dependencies = [ "cfg-if", "js-sys", @@ -3664,13 +3722,24 @@ dependencies = [ "smallvec", ] +[[package]] +name = "group" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5dfbfb3a6cfbd390d5c9564ab283a0349b9b9fcd46a706c1eb10e0db70bfbac7" +dependencies = [ + "ff 0.12.1", + "rand_core", + "subtle", +] + [[package]] name = "group" version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f0f9ef7462f7c099f518d754361858f86d8a07af53ba9af0fe635bbccb151a63" dependencies = [ - "ff", + "ff 0.13.0", "rand_core", "subtle", ] @@ -3912,33 +3981,19 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.23.2" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1788965e61b367cd03a62950836d5cd41560c3577d90e40e0819373194d1661c" -dependencies = [ - "http", - "hyper", - "log", - "rustls 0.20.9", - "rustls-native-certs", - "tokio", - "tokio-rustls 0.23.4", -] - -[[package]] -name = "hyper-rustls" -version = "0.24.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d78e1e73ec14cf7375674f74d7dde185c8206fd9dea6fb6295e8a98098aaa97" +checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" dependencies = [ "futures-util", "http", "hyper", "log", - "rustls 0.21.7", + "rustls", "rustls-native-certs", "tokio", - "tokio-rustls 0.24.1", + "tokio-rustls", + "webpki-roots 0.25.2", ] [[package]] @@ -4304,7 +4359,7 @@ checksum = "6971da4d9c3aa03c3d8f3ff0f4155b534aad021292003895a469716b2a230378" dependencies = [ "base64 0.21.4", "pem 1.1.1", - "ring", + "ring 0.16.20", "serde", "serde_json", "simple_asn1", @@ -4675,12 +4730,14 @@ dependencies = [ [[package]] name = "madsim-aws-sdk-s3" -version = "0.2.25+0.28" +version = "0.3.0+0.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84ab622f10406b583fb75af6f94811d5ae003c1197b9fbaec011ff1d323fc9b6" +checksum = "7f484435d52fc8f9af55a687b7777fcdd8fd9acc01901526367137fc39473c56" dependencies = [ "aws-sdk-s3", "aws-smithy-http", + "aws-smithy-runtime-api", + "aws-smithy-types", "aws-types", "bytes", "http", @@ -5461,7 +5518,7 @@ dependencies = [ "itertools 0.10.5", "log", "oauth2", - "p256", + "p256 0.13.2", "p384", "rand", "rsa", @@ -5701,14 +5758,25 @@ version = "3.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c1b04fb49957986fdce4d6ee7a65027d55d4b6d2265e5848bbb507b58ccfdb6f" +[[package]] +name = "p256" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "51f44edd08f51e2ade572f141051021c5af22677e42b7dd28a88155151c33594" +dependencies = [ + "ecdsa 0.14.8", + "elliptic-curve 0.12.3", + "sha2", +] + [[package]] name = "p256" version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c9863ad85fa8f4460f9c48cb909d38a0d689dba1f6f6988a5e3e0d31071bcd4b" dependencies = [ - "ecdsa", - "elliptic-curve", + "ecdsa 0.16.8", + "elliptic-curve 0.13.6", "primeorder", "sha2", ] @@ -5719,8 +5787,8 @@ version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "70786f51bcc69f6a4c0360e063a4cac5419ef7c5cd5b3c99ad70f3be5ba79209" dependencies = [ - "ecdsa", - "elliptic-curve", + "ecdsa 0.16.8", + "elliptic-curve 0.13.6", "primeorder", "sha2", ] @@ -6064,9 +6132,19 @@ version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c8ffb9f10fa047879315e6625af03c164b16962a5368d724ed16323b68ace47f" dependencies = [ - "der", - "pkcs8", - "spki", + "der 0.7.8", + "pkcs8 0.10.2", + "spki 0.7.2", +] + +[[package]] +name = "pkcs8" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9eca2c590a5f85da82668fa685c09ce2888b9430e83299debf1f34b65fd4a4ba" +dependencies = [ + "der 0.6.1", + "spki 0.6.0", ] [[package]] @@ -6075,8 +6153,8 @@ version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f950b2377845cebe5cf8b5165cb3cc1a5e0fa5cfa3e1f7f55707d8fd82e0a7b7" dependencies = [ - "der", - "spki", + "der 0.7.8", + "spki 0.7.2", ] [[package]] @@ -6321,7 +6399,7 @@ version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3c2fcef82c0ec6eefcc179b978446c399b3cdf73c392c35604e399eee6df1ee3" dependencies = [ - "elliptic-curve", + "elliptic-curve 0.13.6", ] [[package]] @@ -7000,7 +7078,7 @@ dependencies = [ "http", "http-body", "hyper", - "hyper-rustls 0.24.1", + "hyper-rustls", "hyper-tls", "ipnet", "js-sys", @@ -7010,7 +7088,7 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.21.7", + "rustls", "rustls-native-certs", "rustls-pemfile", "serde", @@ -7018,7 +7096,7 @@ dependencies = [ "serde_urlencoded", "tokio", "tokio-native-tls", - "tokio-rustls 0.24.1", + "tokio-rustls", "tokio-util", "tower-service", "url", @@ -7030,6 +7108,17 @@ dependencies = [ "winreg", ] +[[package]] +name = "rfc6979" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7743f17af12fa0b03b803ba12cd6a8d9483a587e89c69445e3909655c0b9fabb" +dependencies = [ + "crypto-bigint 0.4.9", + "hmac", + "zeroize", +] + [[package]] name = "rfc6979" version = "0.4.0" @@ -7059,11 +7148,25 @@ dependencies = [ "libc", "once_cell", "spin 0.5.2", - "untrusted", + "untrusted 0.7.1", "web-sys", "winapi", ] +[[package]] +name = "ring" +version = "0.17.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb0205304757e5d899b9c2e448b867ffd03ae7f988002e47cd24954391394d0b" +dependencies = [ + "cc", + "getrandom", + "libc", + "spin 0.9.8", + "untrusted 0.9.0", + "windows-sys 0.48.0", +] + [[package]] name = "risedev" version = "1.3.0-alpha" @@ -7179,6 +7282,7 @@ dependencies = [ "aws-config", "aws-sdk-s3", "aws-smithy-http", + "aws-smithy-types", "bcc", "bytes", "bytesize", @@ -7506,6 +7610,8 @@ dependencies = [ "aws-sdk-kinesis", "aws-sdk-s3", "aws-smithy-http", + "aws-smithy-runtime-api", + "aws-smithy-types", "aws-types", "base64 0.21.4", "byteorder", @@ -8054,8 +8160,9 @@ dependencies = [ "async-trait", "await-tree", "aws-config", - "aws-smithy-client", "aws-smithy-http", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", "bytes", "crc32fast", @@ -8063,6 +8170,7 @@ dependencies = [ "fail", "futures", "hyper", + "hyper-rustls", "hyper-tls", "itertools 0.11.0", "madsim-aws-sdk-s3", @@ -8070,6 +8178,7 @@ dependencies = [ "opendal", "prometheus", "risingwave_common", + "rustls", "spin 0.9.8", "thiserror", "tokio-retry", @@ -8546,10 +8655,10 @@ dependencies = [ "num-iter", "num-traits", "pkcs1", - "pkcs8", + "pkcs8 0.10.2", "rand_core", - "signature", - "spki", + "signature 2.0.0", + "spki 0.7.2", "subtle", "zeroize", ] @@ -8645,25 +8754,13 @@ dependencies = [ [[package]] name = "rustls" -version = "0.20.9" +version = "0.21.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b80e3dec595989ea8510028f30c408a4630db12c9cbb8de34203b89d6577e99" +checksum = "446e14c5cda4f3f30fe71863c34ec70f5ac79d6087097ad0bb433e1be5edf04c" dependencies = [ "log", - "ring", - "sct", - "webpki", -] - -[[package]] -name = "rustls" -version = "0.21.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd8d6c9f025a446bc4d18ad9632e69aec8f287aa84499ee335599fabd20c3fd8" -dependencies = [ - "log", - "ring", - "rustls-webpki 0.101.4", + "ring 0.17.5", + "rustls-webpki 0.101.7", "sct", ] @@ -8694,18 +8791,18 @@ version = "0.100.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e98ff011474fa39949b7e5c0428f9b4937eda7da7848bbb947786b7be0b27dab" dependencies = [ - "ring", - "untrusted", + "ring 0.16.20", + "untrusted 0.7.1", ] [[package]] name = "rustls-webpki" -version = "0.101.4" +version = "0.101.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d93931baf2d282fff8d3a532bbfd7653f734643161b87e3e01e59a04439bf0d" +checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" dependencies = [ - "ring", - "untrusted", + "ring 0.17.5", + "untrusted 0.9.0", ] [[package]] @@ -8793,8 +8890,8 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d53dcdb7c9f8158937a7981b48accfd39a43af418591a5d008c7b22b5e1b7ca4" dependencies = [ - "ring", - "untrusted", + "ring 0.16.20", + "untrusted 0.7.1", ] [[package]] @@ -8974,16 +9071,30 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "sec1" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3be24c1842290c45df0a7bf069e0c268a747ad05a192f2fd7dcfdbc1cba40928" +dependencies = [ + "base16ct 0.1.1", + "der 0.6.1", + "generic-array", + "pkcs8 0.9.0", + "subtle", + "zeroize", +] + [[package]] name = "sec1" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3e97a565f76233a6003f9f5c54be1d9c5bdfa3eccfb189469f11ec4901c47dc" dependencies = [ - "base16ct", - "der", + "base16ct 0.2.0", + "der 0.7.8", "generic-array", - "pkcs8", + "pkcs8 0.10.2", "subtle", "zeroize", ] @@ -9336,12 +9447,22 @@ version = "0.27.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c1e303f8205714074f6068773f0e29527e0453937fe837c9717d066635b65f31" dependencies = [ - "pkcs8", + "pkcs8 0.10.2", "rand_core", - "signature", + "signature 2.0.0", "zeroize", ] +[[package]] +name = "signature" +version = "1.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "74233d3b3b2f6d4b006dc19dee745e73e2a6bfb6f93607cd3b02bd5b00797d7c" +dependencies = [ + "digest", + "rand_core", +] + [[package]] name = "signature" version = "2.0.0" @@ -9513,6 +9634,16 @@ dependencies = [ "lock_api", ] +[[package]] +name = "spki" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67cf02bbac7a337dc36e4f5a693db6c21e7863f45070f7064577eb4367a3212b" +dependencies = [ + "base64ct", + "der 0.6.1", +] + [[package]] name = "spki" version = "0.7.2" @@ -9520,7 +9651,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9d1e996ef02c474957d681f1b05213dfb0abab947b446a62d37770b23500184a" dependencies = [ "base64ct", - "der", + "der 0.7.8", ] [[package]] @@ -10306,24 +10437,13 @@ dependencies = [ "rand", ] -[[package]] -name = "tokio-rustls" -version = "0.23.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" -dependencies = [ - "rustls 0.20.9", - "tokio", - "webpki", -] - [[package]] name = "tokio-rustls" version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" dependencies = [ - "rustls 0.21.7", + "rustls", "tokio", ] @@ -10443,7 +10563,7 @@ dependencies = [ "prost 0.11.9", "rustls-pemfile", "tokio", - "tokio-rustls 0.24.1", + "tokio-rustls", "tokio-stream", "tower", "tower-layer", @@ -10793,6 +10913,12 @@ version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a156c684c91ea7d62626509bce3cb4e1d9ed5c4d978f7b4352658f96a4c26b4a" +[[package]] +name = "untrusted" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" + [[package]] name = "url" version = "2.4.1" @@ -11016,16 +11142,6 @@ dependencies = [ "wasm-bindgen", ] -[[package]] -name = "webpki" -version = "0.22.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07ecc0cd7cac091bf682ec5efa18b1cff79d617b84181f38b3951dbe135f607f" -dependencies = [ - "ring", - "untrusted", -] - [[package]] name = "webpki-roots" version = "0.23.1" @@ -11279,8 +11395,11 @@ dependencies = [ "async-std", "auto_enums", "aws-credential-types", + "aws-runtime", "aws-sdk-s3", - "aws-smithy-client", + "aws-sigv4", + "aws-smithy-runtime", + "aws-smithy-types", "base64 0.21.4", "bit-vec", "bitflags 2.4.0", @@ -11358,9 +11477,9 @@ dependencies = [ "regex-automata 0.4.1", "regex-syntax 0.8.0", "reqwest", - "ring", + "ring 0.16.20", "rust_decimal", - "rustls 0.21.7", + "rustls", "scopeguard", "sea-orm", "sea-query", @@ -11469,11 +11588,11 @@ dependencies = [ "futures", "http", "hyper", - "hyper-rustls 0.24.1", + "hyper-rustls", "itertools 0.10.5", "log", "percent-encoding", - "rustls 0.21.7", + "rustls", "rustls-pemfile", "seahash", "serde", @@ -11558,3 +11677,8 @@ dependencies = [ "libc", "pkg-config", ] + +[[patch.unused]] +name = "getrandom" +version = "0.2.9" +source = "git+https://github.com/madsim-rs/getrandom.git?rev=8daf97e#8daf97e4142635fe28543b2db9022f5e2544bb5c" diff --git a/Cargo.toml b/Cargo.toml index ee4d8b06774ed..7820fa06cba81 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -68,37 +68,35 @@ repository = "https://github.com/risingwavelabs/risingwave" [workspace.dependencies] await-tree = "0.1.1" -aws-config = { version = "0.55", default-features = false, features = [ +aws-config = { version = "0.57", default-features = false, features = [ "rt-tokio", - "native-tls", + "rustls", ] } -aws-credential-types = { version = "0.55", default-features = false, features = [ +aws-credential-types = { version = "0.57", default-features = false, features = [ "hardcoded-credentials", ] } -aws-sdk-kinesis = { version = "0.28", default-features = false, features = [ +aws-sdk-kinesis = { version = "0.35", default-features = false, features = [ "rt-tokio", - "native-tls", + "rustls", ] } -aws-sdk-s3 = { version = "0.28", default-features = false, features = [ +aws-sdk-s3 = { version = "0.35", default-features = false, features = [ "rt-tokio", - "native-tls", + "rustls", ] } -aws-sdk-ec2 = { version = "0.28", default-features = false, features = [ +aws-sdk-ec2 = { version = "0.35", default-features = false, features = [ "rt-tokio", - "native-tls", + "rustls", ] } -aws-sdk-sqs = { version = "0.28", default-features = false, features = [ +aws-sdk-sqs = { version = "0.35", default-features = false, features = [ "rt-tokio", - "native-tls", + "rustls", ] } -aws-smithy-client = { version = "0.55", default-features = false, features = [ - "rt-tokio", - "native-tls", -] } -aws-smithy-http = "0.55" -aws-smithy-types = "0.55" -aws-endpoint = "0.55" -aws-types = "0.55" +aws-smithy-http = "0.57" +aws-smithy-runtime-api = "0.57" +aws-smithy-runtime = "0.57" +aws-smithy-types = { version = "0.57", default-features = false, features = ["hyper-0-14-x"] } +aws-endpoint = "0.57" +aws-types = "0.57" etcd-client = { package = "madsim-etcd-client", version = "0.4" } futures-async-stream = "0.2.9" hytra = "0.1" @@ -114,7 +112,9 @@ criterion = { version = "0.5", features = ["async_futures"] } tonic = { package = "madsim-tonic", version = "0.4.0" } tonic-build = { package = "madsim-tonic-build", version = "0.4.2" } prost = { version = "0.12" } -icelake = { git = "https://github.com/icelake-io/icelake", rev = "5cdcdffd24f4624a0a43f92c5f368988169a799b", features = ["prometheus"] } +icelake = { git = "https://github.com/icelake-io/icelake", rev = "5cdcdffd24f4624a0a43f92c5f368988169a799b", features = [ + "prometheus", +] } arrow-array = "48" arrow-cast = "48" arrow-schema = "48" diff --git a/src/bench/Cargo.toml b/src/bench/Cargo.toml index 31b8f5ce7cc94..5227c39272c38 100644 --- a/src/bench/Cargo.toml +++ b/src/bench/Cargo.toml @@ -12,6 +12,7 @@ async-trait = "0.1" aws-config = { workspace = true } aws-sdk-s3 = { workspace = true } aws-smithy-http = { workspace = true } +aws-smithy-types = { workspace = true } bcc = { version = "0.0.33", optional = true } bytes = "1" bytesize = { version = "1", features = ["serde"] } diff --git a/src/bench/s3_bench/main.rs b/src/bench/s3_bench/main.rs index f6cbb7cf89200..f0b5d99bd9ff2 100644 --- a/src/bench/s3_bench/main.rs +++ b/src/bench/s3_bench/main.rs @@ -21,7 +21,7 @@ use std::time::{Duration, Instant}; use aws_config::retry::RetryConfig; use aws_sdk_s3::types::{CompletedMultipartUpload, CompletedPart}; use aws_sdk_s3::Client; -use aws_smithy_http::body::SdkBody; +use aws_smithy_types::body::SdkBody; use bytesize::ByteSize; use clap::Parser; use futures::stream::{self, StreamExt}; diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 3e538d7f2eebd..d336b482d4f97 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -32,6 +32,8 @@ aws-credential-types = { workspace = true } aws-sdk-kinesis = { workspace = true } aws-sdk-s3 = { workspace = true } aws-smithy-http = { workspace = true } +aws-smithy-runtime-api = { workspace = true } +aws-smithy-types = { workspace = true } aws-types = { workspace = true } base64 = "0.21" byteorder = "1" diff --git a/src/connector/src/aws_auth.rs b/src/connector/src/aws_auth.rs index d5c6e3fc6b971..cf31744c82e45 100644 --- a/src/connector/src/aws_auth.rs +++ b/src/connector/src/aws_auth.rs @@ -114,7 +114,8 @@ impl AwsAuthProps { if let Some(id) = &self.external_id { role = role.external_id(id); } - Ok(SharedCredentialsProvider::new(role.build(credential))) + let provider = role.build_from_provider(credential).await; + Ok(SharedCredentialsProvider::new(provider)) } else { Ok(credential) } diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index b1e368a2b409e..8a9052c6e4511 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -19,10 +19,12 @@ use anyhow::{anyhow, Result}; use async_trait::async_trait; use aws_sdk_s3::client as s3_client; use aws_sdk_s3::operation::get_object::GetObjectError; -use aws_smithy_http::byte_stream::ByteStream; -use aws_smithy_http::result::SdkError; -use futures::TryStreamExt; +use aws_smithy_http::futures_stream_adapter::FuturesStreamCompatByteStream; +use aws_smithy_runtime_api::client::result::SdkError; +use aws_smithy_types::body::SdkBody; +use aws_smithy_types::byte_stream::ByteStream; use futures_async_stream::try_stream; +use hyper::Response; use io::StreamReader; use risingwave_common::error::RwError; use tokio::io::BufReader; @@ -91,9 +93,10 @@ impl S3FileReader { } }; - let stream_reader = StreamReader::new( - byte_stream.map_err(|e| std::io::Error::new(std::io::ErrorKind::Other, e)), - ); + // FYI: https://github.com/awslabs/smithy-rs/pull/2983 + let byte_stream = FuturesStreamCompatByteStream::new(byte_stream); + + let stream_reader = StreamReader::new(byte_stream); let reader = pin!(BufReader::new(stream_reader)); @@ -142,7 +145,7 @@ impl S3FileReader { bucket_name: &str, object_name: &str, start: usize, - ) -> std::result::Result> { + ) -> std::result::Result>> { let range = if start == 0 { None } else { diff --git a/src/connector/src/source/kinesis/enumerator/client.rs b/src/connector/src/source/kinesis/enumerator/client.rs index 59d1aca06e4f1..8394fae6e8f43 100644 --- a/src/connector/src/source/kinesis/enumerator/client.rs +++ b/src/connector/src/source/kinesis/enumerator/client.rs @@ -75,7 +75,7 @@ impl SplitEnumerator for KinesisSplitEnumerator { Ok(shard_collect .into_iter() .map(|x| KinesisSplit { - shard_id: x.shard_id().unwrap_or_default().to_string().into(), + shard_id: x.shard_id().to_string().into(), // handle start with position in reader part start_position: KinesisOffset::None, end_position: KinesisOffset::None, diff --git a/src/connector/src/source/kinesis/source/message.rs b/src/connector/src/source/kinesis/source/message.rs index a4daa7f1195bd..9e63c05be0e0f 100644 --- a/src/connector/src/source/kinesis/source/message.rs +++ b/src/connector/src/source/kinesis/source/message.rs @@ -40,9 +40,9 @@ impl KinesisMessage { pub fn new(shard_id: SplitId, message: Record) -> Self { KinesisMessage { shard_id, - sequence_number: message.sequence_number.unwrap(), - partition_key: message.partition_key.unwrap(), - payload: message.data.unwrap().into_inner(), + sequence_number: message.sequence_number, + partition_key: message.partition_key, + payload: message.data.into_inner(), } } } diff --git a/src/connector/src/source/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index 4ed00b88a58b2..8ba465342c2ce 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -137,7 +137,7 @@ impl CommonSplitReader for KinesisSplitReader { match self.get_records().await { Ok(resp) => { self.shard_iter = resp.next_shard_iterator().map(String::from); - let chunk = (resp.records().unwrap().iter()) + let chunk = (resp.records().iter()) .map(|r| { SourceMessage::from(KinesisMessage::new( self.shard_id.clone(), diff --git a/src/meta/src/rpc/cloud_provider.rs b/src/meta/src/rpc/cloud_provider.rs index 2b307f955e873..bcf8369759b40 100644 --- a/src/meta/src/rpc/cloud_provider.rs +++ b/src/meta/src/rpc/cloud_provider.rs @@ -62,14 +62,12 @@ impl AwsEc2Client { ) })?; - if let Some(ret) = output.unsuccessful() { - if !ret.is_empty() { - return Err(MetaError::from(anyhow!( - "Failed to delete VPC endpoint {}, error: {:?}", - vpc_endpoint_id, - ret - ))); - } + if !output.unsuccessful().is_empty() { + return Err(MetaError::from(anyhow!( + "Failed to delete VPC endpoint {}, error: {:?}", + vpc_endpoint_id, + output.unsuccessful() + ))); } Ok(()) } @@ -325,13 +323,11 @@ impl AwsEc2Client { let endpoint = output.vpc_endpoint().unwrap(); let mut dns_names = Vec::new(); - if let Some(dns_entries) = endpoint.dns_entries() { - dns_entries.iter().for_each(|e| { - if let Some(dns_name) = e.dns_name() { - dns_names.push(dns_name.to_string()); - } - }); - } + endpoint.dns_entries().iter().for_each(|e| { + if let Some(dns_name) = e.dns_name() { + dns_names.push(dns_name.to_string()); + } + }); Ok(( endpoint.vpc_endpoint_id().unwrap_or_default().to_string(), diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index 2459bf83b5af6..87fd07be7810a 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -12,9 +12,10 @@ repository = { workspace = true } async-trait = "0.1" await-tree = { workspace = true } aws-config = { workspace = true } -aws-sdk-s3 = { version = "0.2", package = "madsim-aws-sdk-s3" } -aws-smithy-client = { workspace = true } +aws-sdk-s3 = { version = "0.3", package = "madsim-aws-sdk-s3" } aws-smithy-http = { workspace = true } +aws-smithy-runtime = { workspace = true } +aws-smithy-runtime-api = { workspace = true } aws-smithy-types = { workspace = true } bytes = { version = "1", features = ["serde"] } crc32fast = "1.3.2" @@ -22,11 +23,13 @@ either = "1" fail = "0.5" futures = { version = "0.3", default-features = false, features = ["alloc"] } hyper = "0.14" +hyper-rustls = { version = "0.24.2", features = ["webpki-roots"] } hyper-tls = "0.5.0" itertools = "0.11" opendal = "0.41" prometheus = { version = "0.13", features = ["process"] } risingwave_common = { workspace = true } +rustls = "0.21.8" spin = "0.9" thiserror = "1" tokio = { version = "0.2", package = "madsim-tokio", features = ["fs"] } diff --git a/src/object_store/src/object/error.rs b/src/object_store/src/object/error.rs index 00877688347ff..7a432a45fdc57 100644 --- a/src/object_store/src/object/error.rs +++ b/src/object_store/src/object/error.rs @@ -20,6 +20,8 @@ use aws_sdk_s3::error::DisplayErrorContext; use aws_sdk_s3::operation::get_object::GetObjectError; use aws_sdk_s3::operation::head_object::HeadObjectError; use aws_sdk_s3::primitives::ByteStreamError; +use aws_smithy_types::body::SdkBody; +use hyper::Response; use risingwave_common::error::BoxedError; use thiserror::Error; use tokio::sync::oneshot::error::RecvError; @@ -79,13 +81,19 @@ impl ObjectError { pub fn is_object_not_found_error(&self) -> bool { match &self.inner { ObjectErrorInner::S3(e) => { - if let Some(aws_smithy_http::result::SdkError::ServiceError(err)) = - e.downcast_ref::>() + if let Some(aws_smithy_runtime_api::client::result::SdkError::ServiceError(err)) = e + .downcast_ref::, + >>() { return matches!(err.err(), GetObjectError::NoSuchKey(_)); } - if let Some(aws_smithy_http::result::SdkError::ServiceError(err)) = - e.downcast_ref::>() + if let Some(aws_smithy_runtime_api::client::result::SdkError::ServiceError(err)) = e + .downcast_ref::, + >>() { return matches!(err.err(), HeadObjectError::NotFound(_)); } diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 71592f45b4d2b..bc826cbbd2ec8 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -30,15 +30,16 @@ use aws_sdk_s3::types::{ CompletedPart, Delete, ExpirationStatus, LifecycleRule, LifecycleRuleFilter, ObjectIdentifier, }; use aws_sdk_s3::Client; -use aws_smithy_client::http_connector::{ConnectorSettings, HttpConnector}; -use aws_smithy_http::body::SdkBody; -use aws_smithy_http::result::SdkError; +use aws_smithy_runtime::client::http::hyper_014::HyperClientBuilder; +use aws_smithy_runtime_api::client::http::HttpClient; +use aws_smithy_runtime_api::client::result::SdkError; +use aws_smithy_types::body::SdkBody; use aws_smithy_types::retry::RetryConfig; use either::Either; use fail::fail_point; use futures::future::{try_join_all, BoxFuture, FutureExt}; use futures::{stream, Stream}; -use hyper::Body; +use hyper::{Body, Response}; use itertools::Itertools; use risingwave_common::config::default::s3_objstore_config; use risingwave_common::monitor::connection::monitor_connector; @@ -495,7 +496,7 @@ impl ObjectStore for S3ObjectStore { // Create identifiers from paths. let mut obj_ids = Vec::with_capacity(slice.len()); for path in slice { - obj_ids.push(ObjectIdentifier::builder().key(path).build()); + obj_ids.push(ObjectIdentifier::builder().key(path).build().unwrap()); } // Build and submit request to delete objects. @@ -504,12 +505,12 @@ impl ObjectStore for S3ObjectStore { .client .delete_objects() .bucket(&self.bucket) - .delete(delete_builder.build()).send() + .delete(delete_builder.build().unwrap()).send() .await?; // Check if there were errors. - if let Some(err_list) = delete_output.errors() && !err_list.is_empty() { - return Err(ObjectError::internal(format!("DeleteObjects request returned exception for some objects: {:?}", err_list))); + if !delete_output.errors().is_empty() { + return Err(ObjectError::internal(format!("DeleteObjects request returned exception for some objects: {:?}", delete_output.errors()))); } } @@ -537,41 +538,37 @@ impl S3ObjectStore { Self::new_with_config(bucket, metrics, S3ObjectStoreConfig::default()).await } - pub fn new_http_connector(config: &S3ObjectStoreConfig) -> impl Into { - // Customize http connector to set keepalive. - let native_tls = { - let mut tls = hyper_tls::native_tls::TlsConnector::builder(); - let tls = tls - .min_protocol_version(Some(hyper_tls::native_tls::Protocol::Tlsv12)) - .build() - .unwrap_or_else(|e| panic!("Error while creating TLS connector: {}", e)); - let mut http = hyper::client::HttpConnector::new(); + pub fn new_http_client(config: &S3ObjectStoreConfig) -> impl HttpClient { + let mut http = hyper::client::HttpConnector::new(); - // connection config - if let Some(keepalive_ms) = config.keepalive_ms.as_ref() { - http.set_keepalive(Some(Duration::from_millis(*keepalive_ms))); - } + // connection config + if let Some(keepalive_ms) = config.keepalive_ms.as_ref() { + http.set_keepalive(Some(Duration::from_millis(*keepalive_ms))); + } - if let Some(nodelay) = config.nodelay.as_ref() { - http.set_nodelay(*nodelay); - } + if let Some(nodelay) = config.nodelay.as_ref() { + http.set_nodelay(*nodelay); + } - if let Some(recv_buffer_size) = config.recv_buffer_size.as_ref() { - http.set_recv_buffer_size(Some(*recv_buffer_size)); - } + if let Some(recv_buffer_size) = config.recv_buffer_size.as_ref() { + http.set_recv_buffer_size(Some(*recv_buffer_size)); + } - if let Some(send_buffer_size) = config.send_buffer_size.as_ref() { - http.set_send_buffer_size(Some(*send_buffer_size)); - } + if let Some(send_buffer_size) = config.send_buffer_size.as_ref() { + http.set_send_buffer_size(Some(*send_buffer_size)); + } - http.enforce_http(false); - hyper_tls::HttpsConnector::from((http, tls.into())) - }; + http.enforce_http(false); + + let conn = hyper_rustls::HttpsConnectorBuilder::new() + .with_webpki_roots() + .https_or_http() + .enable_all_versions() + .wrap_connector(http); - aws_smithy_client::hyper_ext::Adapter::builder() - .hyper_builder(hyper::client::Builder::default()) - .connector_settings(ConnectorSettings::builder().build()) - .build(monitor_connector(native_tls, "S3")) + let conn = monitor_connector(conn, "S3"); + + HyperClientBuilder::new().build(conn) } pub async fn new_with_config( @@ -581,7 +578,7 @@ impl S3ObjectStore { ) -> Self { let sdk_config_loader = aws_config::from_env() .retry_config(RetryConfig::standard().with_max_attempts(4)) - .http_connector(Self::new_http_connector(&config)); + .http_client(Self::new_http_client(&config)); // Retry 3 times if we get server-side errors or throttling errors let client = match std::env::var("RW_S3_ENDPOINT") { @@ -643,7 +640,7 @@ impl S3ObjectStore { let builder = aws_sdk_s3::config::Builder::from(&aws_config::ConfigLoader::default().load().await) .force_path_style(true) - .http_connector(Self::new_http_connector(&S3ObjectStoreConfig::default())); + .http_client(Self::new_http_client(&S3ObjectStoreConfig::default())); let config = builder .region(Region::new("custom")) .endpoint_url(format!("{}{}", endpoint_prefix, address)) @@ -654,6 +651,7 @@ impl S3ObjectStore { )) .build(); let client = Client::from_conf(config); + Self { client, bucket: bucket.to_string(), @@ -718,27 +716,28 @@ impl S3ObjectStore { let mut is_expiration_configured = false; if let Ok(config) = &get_config_result { - for rule in config.rules().unwrap_or_default() { - // Check if the filter is not set or the prifix in the filter is data directory in RisingWave, - // and if the expiration status of the rule is "Enabled". - // If both conditions are met, it is considered that there is a risk of data deletion. - match rule.filter().as_ref() { - Some(&LifecycleRuleFilter::Prefix(prefix)) => { - if let Some(ExpirationStatus::Enabled) = rule.status && data_directory.starts_with(prefix){ - is_expiration_configured = true; - } - } - None => { - if let Some(ExpirationStatus::Enabled) = rule.status { - is_expiration_configured = true; + for rule in config.rules() { + // When both of the conditions are met, it is considered that there is a risk of data deletion. + // + // 1. expiration status rule is enabled + // 2. (a) prefix filter is not set + // or (b) prefix filter is set to the data directory of RisingWave. + // + // P.S. 1 && (2a || 2b) + is_expiration_configured |= rule.status == ExpirationStatus::Enabled // 1 + && match rule.filter().as_ref() { + // 2a + None => true, + // 2b + Some(LifecycleRuleFilter::Prefix(prefix)) + if data_directory.starts_with(prefix) => + { + true } - } - _ => { + _ => false, + }; - } - } - - if matches!(rule.status().unwrap(), ExpirationStatus::Enabled) + if matches!(rule.status(), ExpirationStatus::Enabled) && rule.abort_incomplete_multipart_upload().is_some() { configured_rules.push(rule); @@ -762,10 +761,12 @@ impl S3ObjectStore { .days_after_initiation(S3_INCOMPLETE_MULTIPART_UPLOAD_RETENTION_DAYS) .build(), ) - .build(); + .build() + .unwrap(); let bucket_lifecycle_config = BucketLifecycleConfiguration::builder() .rules(bucket_lifecycle_rule) - .build(); + .build() + .unwrap(); if self .client .put_bucket_lifecycle_configuration() @@ -794,7 +795,9 @@ impl S3ObjectStore { } #[inline(always)] - fn should_retry(err: &Either, ByteStreamError>) -> bool { + fn should_retry( + err: &Either>, ByteStreamError>, + ) -> bool { match err { Either::Left(err) => { if let SdkError::DispatchFailure(e) = err { @@ -870,7 +873,10 @@ struct S3ObjectIter { send_future: Option< BoxFuture< 'static, - Result<(Vec, Option, bool), SdkError>, + Result< + (Vec, Option, bool), + SdkError>, + >, >, >, } @@ -927,7 +933,6 @@ impl Stream for S3ObjectIter { Ok(r) => { let more = r .contents() - .unwrap_or_default() .iter() .map(|obj| ObjectMetadata { key: obj.key().expect("key required").to_owned(), @@ -950,8 +955,8 @@ impl Stream for S3ObjectIter { } } -impl From, ByteStreamError>> for ObjectError { - fn from(e: Either, ByteStreamError>) -> Self { +impl From>, ByteStreamError>> for ObjectError { + fn from(e: Either>, ByteStreamError>) -> Self { match e { Either::Left(e) => e.into(), Either::Right(e) => e.into(), diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index 15ae2e667ff6c..c2e37b78e14a1 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -13,7 +13,7 @@ normal = ["serde"] [dependencies] anyhow = "1.0" async-trait = "0.1" -aws-sdk-s3 = { version = "0.2", package = "madsim-aws-sdk-s3" } +aws-sdk-s3 = { version = "0.3", package = "madsim-aws-sdk-s3" } cfg-or-panic = "0.2" clap = { version = "4", features = ["derive"] } console = "0.15" diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index fe483dbc7b1a0..b263a9c5bcefb 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -22,9 +22,12 @@ ahash = { version = "0.8" } allocator-api2 = { version = "0.2", default-features = false, features = ["alloc", "nightly"] } anyhow = { version = "1", features = ["backtrace"] } async-std = { version = "1", features = ["attributes", "tokio1"] } -aws-credential-types = { version = "0.55", default-features = false, features = ["hardcoded-credentials"] } -aws-sdk-s3 = { version = "0.28", features = ["native-tls"] } -aws-smithy-client = { version = "0.55", default-features = false, features = ["native-tls", "rustls"] } +aws-credential-types = { version = "0.57", default-features = false, features = ["hardcoded-credentials"] } +aws-runtime = { version = "0.57", default-features = false, features = ["event-stream", "sigv4a"] } +aws-sdk-s3 = { version = "0.35" } +aws-sigv4 = { version = "0.57", features = ["http0-compat", "sign-eventstream", "sigv4a"] } +aws-smithy-runtime = { version = "0.57", default-features = false, features = ["client", "rt-tokio", "tls-rustls"] } +aws-smithy-types = { version = "0.57", default-features = false, features = ["byte-stream-poll-next", "http-body-0-4-x", "hyper-0-14-x", "rt-tokio"] } base64 = { version = "0.21", features = ["alloc"] } bit-vec = { version = "0.6" } bitflags = { version = "2", default-features = false, features = ["serde", "std"] } @@ -132,7 +135,7 @@ toml_datetime = { version = "0.6", default-features = false, features = ["serde" toml_edit = { version = "0.19", features = ["serde"] } tonic-274715c4dabd11b0 = { package = "tonic", version = "0.9", features = ["gzip", "tls-webpki-roots"] } tonic-93f6ce9d446188ac = { package = "tonic", version = "0.10" } -tower = { version = "0.4", features = ["balance", "buffer", "filter", "limit", "load-shed", "retry", "timeout", "util"] } +tower = { version = "0.4", features = ["balance", "buffer", "filter", "limit", "load-shed", "timeout", "util"] } tracing = { version = "0.1", features = ["log"] } tracing-core = { version = "0.1" } tracing-subscriber = { version = "0.3", features = ["env-filter", "json", "local-time", "parking_lot"] } From 3b7036ca19638206d94b986163170462a65a9fb0 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 14 Nov 2023 15:20:49 +0800 Subject: [PATCH 66/77] feat(cdc): persist the backfill state for table-on-source (#13276) --- ci/scripts/e2e-source-test.sh | 10 +- e2e_test/source/cdc/cdc.check_new_rows.slt | 18 ++ e2e_test/source/cdc/cdc.share_stream.slt | 9 +- .../source/SourceValidateHandler.java | 33 +-- .../source/common/DbzConnectorConfig.java | 50 +++-- proto/plan_common.proto | 2 + src/common/src/catalog/external_table.rs | 6 +- src/compute/tests/cdc_tests.rs | 3 +- src/connector/src/parser/mod.rs | 14 +- src/connector/src/source/external.rs | 6 +- src/frontend/src/handler/create_table.rs | 4 +- .../plan_node/stream_cdc_table_scan.rs | 93 +++----- src/meta/service/src/ddl_service.rs | 119 ++++++----- src/meta/src/manager/catalog/mod.rs | 5 +- .../src/executor/backfill/cdc/cdc_backfill.rs | 139 +++++++----- src/stream/src/executor/backfill/cdc/mod.rs | 2 +- src/stream/src/executor/backfill/cdc/state.rs | 202 +++++++++++++++--- src/stream/src/executor/dispatch.rs | 2 +- .../src/from_proto/source/trad_source.rs | 3 +- src/stream/src/from_proto/stream_scan.rs | 17 +- 20 files changed, 463 insertions(+), 274 deletions(-) diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 2f9c29d76b957..81cb5988926cf 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -71,10 +71,18 @@ export MYSQL_HOST=mysql MYSQL_TCP_PORT=3306 MYSQL_PWD=123456 sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.share_stream.slt' -# kill cluster and the connector node +# kill cluster cargo make kill echo "cluster killed " +# insert into mytest database (cdc.share_stream.slt) +mysql --protocol=tcp -u root mytest -e "INSERT INTO products + VALUES (default,'RisingWave','Next generation Streaming Database'), + (default,'Materialize','The Streaming Database You Already Know How to Use'); + UPDATE products SET name = 'RW' WHERE id <= 103; + INSERT INTO orders VALUES (default, '2022-12-01 15:08:22', 'Sam', 1000.52, 110, false);" + + # insert new rows mysql --host=mysql --port=3306 -u root -p123456 < ./e2e_test/source/cdc/mysql_cdc_insert.sql psql < ./e2e_test/source/cdc/postgres_cdc_insert.sql diff --git a/e2e_test/source/cdc/cdc.check_new_rows.slt b/e2e_test/source/cdc/cdc.check_new_rows.slt index 2acf7813feebf..58362225660ea 100644 --- a/e2e_test/source/cdc/cdc.check_new_rows.slt +++ b/e2e_test/source/cdc/cdc.check_new_rows.slt @@ -47,3 +47,21 @@ select v1, v2, v3 from mytable order by v1; 2 2 yes 3 3 no 4 4 no + +# shared cdc source +query I +SELECT * from products_test_cnt +---- +11 + +query I +SELECT * from orders_test_cnt +---- +4 + +query ITT +SELECT * FROM products_test order by id limit 3 +---- +101 RW Small 2-wheel scooter +102 RW 12V car battery +103 RW 12-pack of drill bits with sizes ranging from #40 to #3 diff --git a/e2e_test/source/cdc/cdc.share_stream.slt b/e2e_test/source/cdc/cdc.share_stream.slt index 459cde90580bc..5a6342a8cf35c 100644 --- a/e2e_test/source/cdc/cdc.share_stream.slt +++ b/e2e_test/source/cdc/cdc.share_stream.slt @@ -7,6 +7,10 @@ mysql --protocol=tcp -u root -e "DROP DATABASE IF EXISTS mytest; CREATE DATABASE system ok mysql --protocol=tcp -u root mytest < e2e_test/source/cdc/mysql_create.sql +# generate data to mysql +system ok +mysql --protocol=tcp -u root mytest < e2e_test/source/cdc/mysql_init_data.sql + # enable cdc backfill in ci statement ok set cdc_backfill='true'; @@ -47,11 +51,6 @@ create materialized view products_test_cnt as select count(*) as cnt from produc statement ok create materialized view orders_test_cnt as select count(*) as cnt from orders_test; - -# generate data to mysql -system ok -mysql --protocol=tcp -u root mytest < e2e_test/source/cdc/mysql_init_data.sql - sleep 5s # check ingestion results diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/SourceValidateHandler.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/SourceValidateHandler.java index 72b361e04bc0c..4c4cc092fa16d 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/SourceValidateHandler.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/SourceValidateHandler.java @@ -62,8 +62,8 @@ public static ConnectorServiceProto.ValidateSourceResponse validateResponse(Stri .build(); } - public static void ensurePropNotNull(Map props, String name) { - if (!props.containsKey(name)) { + public static void ensurePropNotBlank(Map props, String name) { + if (StringUtils.isBlank(props.get(name))) { throw ValidatorUtils.invalidArgument( String.format("'%s' not found, please check the WITH properties", name)); } @@ -73,39 +73,39 @@ public static void validateSource(ConnectorServiceProto.ValidateSourceRequest re throws Exception { var props = request.getPropertiesMap(); - ensurePropNotNull(props, DbzConnectorConfig.HOST); - ensurePropNotNull(props, DbzConnectorConfig.PORT); - ensurePropNotNull(props, DbzConnectorConfig.DB_NAME); - ensurePropNotNull(props, DbzConnectorConfig.USER); - ensurePropNotNull(props, DbzConnectorConfig.PASSWORD); + ensurePropNotBlank(props, DbzConnectorConfig.HOST); + ensurePropNotBlank(props, DbzConnectorConfig.PORT); + ensurePropNotBlank(props, DbzConnectorConfig.DB_NAME); + ensurePropNotBlank(props, DbzConnectorConfig.USER); + ensurePropNotBlank(props, DbzConnectorConfig.PASSWORD); // ensure table name is passed by user in single mode if (Utils.getCdcSourceMode(props) == CdcSourceMode.SINGLE_MODE) { - ensurePropNotNull(props, DbzConnectorConfig.TABLE_NAME); + ensurePropNotBlank(props, DbzConnectorConfig.TABLE_NAME); } TableSchema tableSchema = TableSchema.fromProto(request.getTableSchema()); switch (request.getSourceType()) { case POSTGRES: - ensurePropNotNull(props, DbzConnectorConfig.TABLE_NAME); - ensurePropNotNull(props, DbzConnectorConfig.PG_SCHEMA_NAME); - ensurePropNotNull(props, DbzConnectorConfig.PG_SLOT_NAME); - ensurePropNotNull(props, DbzConnectorConfig.PG_PUB_NAME); - ensurePropNotNull(props, DbzConnectorConfig.PG_PUB_CREATE); + ensurePropNotBlank(props, DbzConnectorConfig.TABLE_NAME); + ensurePropNotBlank(props, DbzConnectorConfig.PG_SCHEMA_NAME); + ensurePropNotBlank(props, DbzConnectorConfig.PG_SLOT_NAME); + ensurePropNotBlank(props, DbzConnectorConfig.PG_PUB_NAME); + ensurePropNotBlank(props, DbzConnectorConfig.PG_PUB_CREATE); try (var validator = new PostgresValidator(props, tableSchema)) { validator.validateAll(); } break; case CITUS: - ensurePropNotNull(props, DbzConnectorConfig.TABLE_NAME); - ensurePropNotNull(props, DbzConnectorConfig.PG_SCHEMA_NAME); + ensurePropNotBlank(props, DbzConnectorConfig.TABLE_NAME); + ensurePropNotBlank(props, DbzConnectorConfig.PG_SCHEMA_NAME); try (var coordinatorValidator = new CitusValidator(props, tableSchema)) { coordinatorValidator.validateDistributedTable(); coordinatorValidator.validateTable(); } - ensurePropNotNull(props, DbzConnectorConfig.DB_SERVERS); + ensurePropNotBlank(props, DbzConnectorConfig.DB_SERVERS); var workerServers = StringUtils.split(props.get(DbzConnectorConfig.DB_SERVERS), ','); // props extracted from grpc request, clone it to modify @@ -126,6 +126,7 @@ public static void validateSource(ConnectorServiceProto.ValidateSourceRequest re break; case MYSQL: + ensurePropNotBlank(props, DbzConnectorConfig.MYSQL_SERVER_ID); try (var validator = new MySqlValidator(props, tableSchema)) { validator.validateAll(); } diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzConnectorConfig.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzConnectorConfig.java index a77da9b6d2ed0..1cf07e14eca4a 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzConnectorConfig.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzConnectorConfig.java @@ -63,6 +63,7 @@ public class DbzConnectorConfig { private static final String DBZ_PROPERTY_PREFIX = "debezium."; + private static final String SNAPSHOT_MODE_KEY = "debezium.snapshot.mode"; private static final String SNAPSHOT_MODE_BACKFILL = "rw_cdc_backfill"; private static Map extractDebeziumProperties( @@ -103,31 +104,48 @@ public DbzConnectorConfig( String startOffset, Map userProps, boolean snapshotDone) { + + StringSubstitutor substitutor = new StringSubstitutor(userProps); + var dbzProps = initiateDbConfig(DBZ_CONFIG_FILE, substitutor); + var isCdcBackfill = + null != userProps.get(SNAPSHOT_MODE_KEY) + && userProps.get(SNAPSHOT_MODE_KEY).equals(SNAPSHOT_MODE_BACKFILL); + LOG.info( - "DbzConnectorConfig: source={}, sourceId={}, startOffset={}, snapshotDone={}", + "DbzConnectorConfig: source={}, sourceId={}, startOffset={}, snapshotDone={}, isCdcBackfill={}", source, sourceId, startOffset, - snapshotDone); + snapshotDone, + isCdcBackfill); - StringSubstitutor substitutor = new StringSubstitutor(userProps); - var dbzProps = initiateDbConfig(DBZ_CONFIG_FILE, substitutor); if (source == SourceTypeE.MYSQL) { var mysqlProps = initiateDbConfig(MYSQL_CONFIG_FILE, substitutor); - // if snapshot phase is finished and offset is specified, we will continue binlog - // reading from the given offset - if (snapshotDone && null != startOffset && !startOffset.isBlank()) { - // 'snapshot.mode=schema_only_recovery' must be configured if binlog offset is - // specified. It only snapshots the schemas, not the data, and continue binlog - // reading from the specified offset - mysqlProps.setProperty("snapshot.mode", "schema_only_recovery"); - mysqlProps.setProperty( - ConfigurableOffsetBackingStore.OFFSET_STATE_VALUE, startOffset); - } else if (mysqlProps.getProperty("snapshot.mode").equals(SNAPSHOT_MODE_BACKFILL)) { - // only snapshot table schemas which is not required by the source parser - mysqlProps.setProperty("snapshot.mode", "schema_only"); + if (isCdcBackfill) { // disable snapshot locking at all mysqlProps.setProperty("snapshot.locking.mode", "none"); + + // If cdc backfill enabled, the source only emit incremental changes, so we must + // rewind to the given offset and continue binlog reading from there + if (null != startOffset && !startOffset.isBlank()) { + mysqlProps.setProperty("snapshot.mode", "schema_only_recovery"); + mysqlProps.setProperty( + ConfigurableOffsetBackingStore.OFFSET_STATE_VALUE, startOffset); + } else { + // read upstream table schemas and emit incremental changes only + mysqlProps.setProperty("snapshot.mode", "schema_only"); + } + } else { + // if snapshot phase is finished and offset is specified, we will continue binlog + // reading from the given offset + if (snapshotDone && null != startOffset && !startOffset.isBlank()) { + // 'snapshot.mode=schema_only_recovery' must be configured if binlog offset is + // specified. It only snapshots the schemas, not the data, and continue binlog + // reading from the specified offset + mysqlProps.setProperty("snapshot.mode", "schema_only_recovery"); + mysqlProps.setProperty( + ConfigurableOffsetBackingStore.OFFSET_STATE_VALUE, startOffset); + } } dbzProps.putAll(mysqlProps); diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 9f43d9755edfe..4f1eccfab8666 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -84,6 +84,8 @@ message ExternalTableDesc { string table_name = 4; repeated uint32 stream_key = 5; map connect_properties = 6; + // upstream cdc source job id + uint32 source_id = 7; } enum JoinType { diff --git a/src/common/src/catalog/external_table.rs b/src/common/src/catalog/external_table.rs index 28d43a7e97a0a..b006bbd50d362 100644 --- a/src/common/src/catalog/external_table.rs +++ b/src/common/src/catalog/external_table.rs @@ -23,9 +23,12 @@ use crate::util::sort_util::ColumnOrder; /// Compute node will use this information to connect to the external database and scan the table. #[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] pub struct CdcTableDesc { - /// Id of the upstream source in sharing cdc mode + /// Id of the table in RW pub table_id: TableId, + /// Id of the upstream source in sharing cdc mode + pub source_id: TableId, + /// The full name of the table in external database, e.g. `database_name.table.name` in MySQL /// and `schema_name.table_name` in the Postgres. pub external_table_name: String, @@ -58,6 +61,7 @@ impl CdcTableDesc { pub fn to_protobuf(&self) -> ExternalTableDesc { ExternalTableDesc { table_id: self.table_id.into(), + source_id: self.source_id.into(), columns: self.columns.iter().map(Into::into).collect(), pk: self.pk.iter().map(|v| v.to_protobuf()).collect(), table_name: self.external_table_name.clone(), diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index a6e515a1ea571..b692b27973736 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -216,7 +216,8 @@ async fn test_cdc_backfill() -> StreamResult<()> { vec![0, 1, 2], None, Arc::new(StreamingMetrics::unused()), - source_state_handler, + None, + Some(source_state_handler), false, 4, // 4 rows in a snapshot chunk ); diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index cdb3ef7f51620..53e843b5cfdbb 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -543,14 +543,12 @@ async fn into_chunk_stream(mut parser: P, data_stream for (i, msg) in batch.into_iter().enumerate() { if msg.key.is_none() && msg.payload.is_none() { - if parser.parser_format() == ParserFormat::Debezium { - tracing::debug!(offset = msg.offset, "skip parsing of heartbeat message"); - // empty payload means a heartbeat in cdc source - // heartbeat message offset should not overwrite data messages offset - split_offset_mapping - .entry(msg.split_id) - .or_insert(msg.offset.clone()); - } + tracing::debug!(offset = msg.offset, "skip parsing of heartbeat message"); + // assumes an empty message as a heartbeat + // heartbeat message offset should not overwrite data messages offset + split_offset_mapping + .entry(msg.split_id) + .or_insert(msg.offset.clone()); continue; } diff --git a/src/connector/src/source/external.rs b/src/connector/src/source/external.rs index c70f41b32b581..2fa17a0add927 100644 --- a/src/connector/src/source/external.rs +++ b/src/connector/src/source/external.rs @@ -121,7 +121,7 @@ impl SchemaTableName { } } -#[derive(Debug, Clone, Default, PartialEq, PartialOrd)] +#[derive(Debug, Clone, Default, PartialEq, PartialOrd, Serialize, Deserialize)] pub struct MySqlOffset { pub filename: String, pub position: u64, @@ -133,14 +133,14 @@ impl MySqlOffset { } } -#[derive(Debug, Clone, Default, PartialEq, PartialOrd)] +#[derive(Debug, Clone, Default, PartialEq, PartialOrd, Serialize, Deserialize)] pub struct PostgresOffset { pub txid: u64, pub lsn: u64, pub tx_usec: u64, } -#[derive(Debug, Clone, PartialEq, PartialOrd)] +#[derive(Debug, Clone, PartialEq, PartialOrd, Serialize, Deserialize)] pub enum CdcOffset { MySql(MySqlOffset), Postgres(PostgresOffset), diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 5654bc5ed4ed0..b81bc00394bb4 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -562,6 +562,7 @@ pub(crate) async fn gen_create_table_plan_with_source( let cdc_table_desc = CdcTableDesc { table_id: TableId::placeholder(), + source_id: TableId::placeholder(), external_table_name: "".to_string(), pk: table_pk, columns: columns.iter().map(|c| c.column_desc.clone()).collect(), @@ -847,7 +848,8 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( derive_connect_properties(source.as_ref(), external_table_name.clone())?; let cdc_table_desc = CdcTableDesc { - table_id: source.id.into(), // source can be considered as an external table + table_id: TableId::placeholder(), // will be filled in meta node + source_id: source.id.into(), // id of cdc source streaming job external_table_name: external_table_name.clone(), pk: table_pk, columns: columns.iter().map(|c| c.column_desc.clone()).collect(), diff --git a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs index 7b68e9d80c93e..4731227bc54c7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs @@ -15,7 +15,6 @@ use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{ColumnCatalog, Field}; -use risingwave_common::hash::VirtualNode; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; @@ -73,37 +72,9 @@ impl StreamCdcTableScan { StreamScanType::CdcBackfill } - /// Build catalog for backfill state - /// - /// Schema: | vnode | pk ... | `backfill_finished` | `row_count` | - /// - /// key: | vnode | - /// value: | pk ... | `backfill_finished` | `row_count` | - /// - /// When we update the backfill progress, - /// we update it for all vnodes. - /// - /// `pk` refers to the upstream pk which we use to track the backfill progress. - /// - /// `vnode` is the corresponding vnode of the upstream's distribution key. - /// It should also match the vnode of the backfill executor. - /// - /// `backfill_finished` is a boolean which just indicates if backfill is done. - /// - /// `row_count` is a count of rows which indicates the # of rows per executor. - /// We used to track this in memory. - /// But for backfill persistence we have to also persist it. - /// - /// FIXME(kwannoel): - /// - Across all vnodes, the values are the same. - /// - e.g. | vnode | pk ... | `backfill_finished` | `row_count` | - /// | 1002 | Int64(1) | t | 10 | - /// | 1003 | Int64(1) | t | 10 | - /// | 1003 | Int64(1) | t | 10 | - /// Eventually we should track progress per vnode, to support scaling with both mview and - /// the corresponding `no_shuffle_backfill`. - /// However this is not high priority, since we are working on supporting arrangement backfill, - /// which already has this capability. + /// Build catalog for cdc backfill state + /// Right now we only persist whether the backfill is finished and the corresponding cdc offset + /// schema: | `split_id` | `pk...` | `backfill_finished` | `row_count` | `cdc_offset` | pub fn build_backfill_state_catalog( &self, state: &mut BuildFragmentGraphState, @@ -112,9 +83,9 @@ impl StreamCdcTableScan { let mut catalog_builder = TableCatalogBuilder::new(properties); let upstream_schema = &self.core.get_table_columns(); - // We use vnode as primary key in state table. - // If `Distribution::Single`, vnode will just be `VirtualNode::default()`. - catalog_builder.add_column(&Field::with_name(VirtualNode::RW_TYPE, "vnode")); + // Use `split_id` as primary key in state table. + // Currently we only support single split for cdc backfill. + catalog_builder.add_column(&Field::with_name(DataType::Varchar, "split_id")); catalog_builder.add_order_column(0, OrderType::ascending()); // pk columns @@ -123,27 +94,17 @@ impl StreamCdcTableScan { catalog_builder.add_column(&Field::from(col)); } - // `backfill_finished` column - catalog_builder.add_column(&Field::with_name( - DataType::Boolean, - format!("{}_backfill_finished", self.table_name()), - )); + catalog_builder.add_column(&Field::with_name(DataType::Boolean, "backfill_finished")); - // `row_count` column - catalog_builder.add_column(&Field::with_name( - DataType::Int64, - format!("{}_row_count", self.table_name()), - )); + // `row_count` column, the number of rows read from snapshot + catalog_builder.add_column(&Field::with_name(DataType::Int64, "row_count")); - // Reuse the state store pk (vnode) as the vnode as well. - catalog_builder.set_vnode_col_idx(0); - catalog_builder.set_dist_key_in_pk(vec![0]); - - let num_of_columns = catalog_builder.columns().len(); - catalog_builder.set_value_indices((1..num_of_columns).collect_vec()); + // The offset is only for observability, not for recovery right now + catalog_builder.add_column(&Field::with_name(DataType::Jsonb, "cdc_offset")); + // leave dist key empty, since the cdc backfill executor is singleton catalog_builder - .build(vec![0], 1) + .build(vec![], 1) .with_id(state.gen_table_id_wrapped()) } } @@ -250,20 +211,20 @@ impl StreamCdcTableScan { .build_backfill_state_catalog(state) .to_internal_table_prost(); - let node_body = - // don't need batch plan for cdc source - PbNodeBody::StreamScan(StreamScanNode { - table_id: self.core.cdc_table_desc.table_id.table_id, - stream_scan_type: self.stream_scan_type as i32, - // The column indices need to be forwarded to the downstream - output_indices, - upstream_column_ids, - // The table desc used by backfill executor - state_table: Some(catalog), - rate_limit: None, - cdc_table_desc: Some(self.core.cdc_table_desc.to_protobuf()), - ..Default::default() - }); + // We need to pass the id of upstream source job here + let upstream_source_id = self.core.cdc_table_desc.source_id.table_id; + let node_body = PbNodeBody::StreamScan(StreamScanNode { + table_id: upstream_source_id, + stream_scan_type: self.stream_scan_type as i32, + // The column indices need to be forwarded to the downstream + output_indices, + upstream_column_ids, + // The table desc used by backfill executor + state_table: Some(catalog), + rate_limit: None, + cdc_table_desc: Some(self.core.cdc_table_desc.to_protobuf()), + ..Default::default() + }); PbStreamNode { fields: self.schema().to_prost(), diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index c63f81da472c3..abc9e690ceeb9 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use anyhow::anyhow; use rand::Rng; +use risingwave_common::catalog::TableId; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::stream_graph_visitor::visit_fragment; use risingwave_connector::sink::catalog::SinkId; @@ -457,25 +458,23 @@ impl DdlService for DdlServiceImpl { let mut mview = request.materialized_view.unwrap(); let mut fragment_graph = request.fragment_graph.unwrap(); let table_id = self.gen_unique_id::<{ IdCategory::Table }>().await?; + // If we're creating a table with connector, we should additionally fill its ID first. - if let Some(source) = &mut source { + let source_id = if source.is_some() { // Generate source id. - let source_id = self.gen_unique_id::<{ IdCategory::Table }>().await?; // TODO: Use source category - fill_table_source(source, source_id, &mut mview, table_id, &mut fragment_graph); - - // Modify properties for cdc sources if needed - if let Some(connector) = source.properties.get(UPSTREAM_SOURCE_KEY) { - if matches!( - CdcSourceType::from(connector.as_str()), - CdcSourceType::Mysql - ) { - fill_cdc_mysql_server_id(&mut fragment_graph); - } - } - } + self.gen_unique_id::<{ IdCategory::Table }>().await? // TODO: Use source category + } else { + TableId::placeholder().into() + }; - let mut stream_job = StreamingJob::Table(source, mview, job_type); + fill_table_stream_graph_info( + source.as_mut().map(|source| (source, source_id)), + (&mut mview, table_id), + job_type, + &mut fragment_graph, + ); + let mut stream_job = StreamingJob::Table(source, mview, job_type); stream_job.set_id(table_id); let version = self @@ -577,7 +576,12 @@ impl DdlService for DdlServiceImpl { { let source = source.as_mut().unwrap(); let table_id = table.id; - fill_table_source(source, source_id, &mut table, table_id, &mut fragment_graph); + fill_table_stream_graph_info( + Some((source, source_id)), + (&mut table, table_id), + TableJobType::General, + &mut fragment_graph, + ); } let table_col_index_mapping = ColIndexMapping::from_protobuf(&req.table_col_index_mapping.unwrap()); @@ -853,51 +857,60 @@ impl DdlServiceImpl { } } -fn fill_table_source( - source: &mut PbSource, - source_id: u32, - table: &mut PbTable, - table_id: u32, +/// Fill in necessary information for table stream graph. +fn fill_table_stream_graph_info( + mut source_info: Option<(&mut PbSource, u32)>, + table_info: (&mut PbTable, u32), + table_job_type: TableJobType, fragment_graph: &mut PbStreamFragmentGraph, ) { - // If we're creating a table with connector, we should additionally fill its ID first. - source.id = source_id; - - let mut source_count = 0; + let (table, table_id) = table_info; for fragment in fragment_graph.fragments.values_mut() { visit_fragment(fragment, |node_body| { if let NodeBody::Source(source_node) = node_body { - // TODO: Refactor using source id. - source_node.source_inner.as_mut().unwrap().source_id = source_id; - source_count += 1; - } - }); - } - assert_eq!( - source_count, 1, - "require exactly 1 external stream source when creating table with a connector" - ); + // If we're creating a table with connector, we should additionally fill its ID first. + if let Some(&mut (ref mut source, source_id)) = source_info.as_mut() { + source.id = source_id; + let mut source_count = 0; + + source_node.source_inner.as_mut().unwrap().source_id = source_id; + source_count += 1; + + // Generate a random server id for mysql cdc source if needed + // `server.id` (in the range from 1 to 2^32 - 1). This value MUST be unique across whole replication + // group (that is, different from any other server id being used by any master or slave) + if let Some(connector) = source.properties.get(UPSTREAM_SOURCE_KEY) + && matches!(CdcSourceType::from(connector.as_str()),CdcSourceType::Mysql) { + let props = &mut source_node.source_inner.as_mut().unwrap().properties; + let rand_server_id = rand::thread_rng().gen_range(1..u32::MAX); + props + .entry("server.id".to_string()) + .or_insert(rand_server_id.to_string()); + + // make these two `Source` consistent + props.clone_into(&mut source.properties); + } - // Fill in the correct table id for source. - source.optional_associated_table_id = - Some(OptionalAssociatedTableId::AssociatedTableId(table_id)); + assert_eq!( + source_count, 1, + "require exactly 1 external stream source when creating table with a connector" + ); - // Fill in the correct source id for mview. - table.optional_associated_source_id = - Some(OptionalAssociatedSourceId::AssociatedSourceId(source_id)); -} + // Fill in the correct table id for source. + source.optional_associated_table_id = + Some(OptionalAssociatedTableId::AssociatedTableId(table_id)); -// `server.id` (in the range from 1 to 2^32 - 1). This value MUST be unique across whole replication -// group (that is, different from any other server id being used by any master or slave) -fn fill_cdc_mysql_server_id(fragment_graph: &mut PbStreamFragmentGraph) { - for fragment in fragment_graph.fragments.values_mut() { - visit_fragment(fragment, |node_body| { - if let NodeBody::Source(source_node) = node_body { - let props = &mut source_node.source_inner.as_mut().unwrap().properties; - let rand_server_id = rand::thread_rng().gen_range(1..u32::MAX); - props - .entry("server.id".to_string()) - .or_insert(rand_server_id.to_string()); + // Fill in the correct source id for mview. + table.optional_associated_source_id = + Some(OptionalAssociatedSourceId::AssociatedSourceId(source_id)); + } + } + + // fill table id for cdc backfill + if let NodeBody::StreamScan(node) = node_body && table_job_type == TableJobType::SharedCdcSource { + if let Some(table) = node.cdc_table_desc.as_mut() { + table.table_id = table_id; + } } }); } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 04bc743ae5348..9254702073e99 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -1945,7 +1945,7 @@ impl CatalogManager { pub async fn finish_create_source_procedure( &self, mut source: Source, - internal_tables: Vec, + mut internal_tables: Vec
, ) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; @@ -1961,7 +1961,8 @@ impl CatalogManager { source.created_at_epoch = Some(Epoch::now().0); sources.insert(source.id, source.clone()); - for table in &internal_tables { + for table in &mut internal_tables { + table.stream_job_status = PbStreamJobStatus::Created.into(); tables.insert(table.id, table.clone()); } commit_meta!(self, sources, tables)?; diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index b756d48944a03..451577e98a60b 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -34,7 +34,10 @@ use risingwave_connector::source::external::CdcOffset; use risingwave_connector::source::{SourceColumnDesc, SourceContext, SplitMetaData}; use risingwave_storage::StateStore; -use crate::executor::backfill::cdc::state::{CdcBackfillStateImpl, SingleTableState}; +use crate::common::table::state_table::StateTable; +use crate::executor::backfill::cdc::state::{ + CdcBackfillStateImpl, MultiBackfillState, SingleBackfillState, +}; use crate::executor::backfill::upstream_table::external::ExternalStorageTable; use crate::executor::backfill::upstream_table::snapshot::{ SnapshotReadArgs, UpstreamTableRead, UpstreamTableReader, @@ -48,14 +51,17 @@ use crate::executor::{ ExecutorInfo, Message, Mutation, PkIndicesRef, SourceStateTableHandler, StreamExecutorError, StreamExecutorResult, }; -use crate::task::{ActorId, CreateMviewProgress}; +use crate::task::CreateMviewProgress; + +/// `split_id`, `is_finished`, `row_count`, `cdc_offset` all occupy 1 column each. +const METADATA_STATE_LEN: usize = 4; pub struct CdcBackfillExecutor { actor_ctx: ActorContextRef, info: ExecutorInfo, - /// Upstream external table - upstream_table: ExternalStorageTable, + /// The external table to be backfilled + external_table: ExternalStorageTable, /// Upstream changelog stream which may contain metadata columns, e.g. `_rw_offset` upstream: BoxedExecutor, @@ -64,13 +70,14 @@ pub struct CdcBackfillExecutor { /// User may select a subset of columns from the upstream table. output_indices: Vec, - actor_id: ActorId, - /// State table of the Source executor - source_state_handler: SourceStateTableHandler, + source_state_handler: Option>, shared_cdc_source: bool, + /// State table of the CdcBackfill executor + state_table: Option>, + progress: Option, metrics: Arc, @@ -83,24 +90,25 @@ impl CdcBackfillExecutor { pub fn new( actor_ctx: ActorContextRef, info: ExecutorInfo, - upstream_table: ExternalStorageTable, + external_table: ExternalStorageTable, upstream: BoxedExecutor, output_indices: Vec, progress: Option, metrics: Arc, - source_state_handler: SourceStateTableHandler, + state_table: Option>, + source_state_handler: Option>, shared_cdc_source: bool, chunk_size: usize, ) -> Self { Self { actor_ctx, info, - upstream_table, + external_table, upstream, output_indices, - actor_id: 0, source_state_handler, shared_cdc_source, + state_table, progress, metrics, chunk_size, @@ -110,13 +118,13 @@ impl CdcBackfillExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { // The primary key columns, in the output columns of the upstream_table scan. - let pk_in_output_indices = self.upstream_table.pk_in_output_indices().unwrap(); - let pk_order = self.upstream_table.pk_order_types().to_vec(); + let pk_in_output_indices = self.external_table.pk_in_output_indices().unwrap(); + let pk_order = self.external_table.pk_order_types().to_vec(); let shared_cdc_source = self.shared_cdc_source; - let upstream_table_id = self.upstream_table.table_id().table_id; - let upstream_table_schema = self.upstream_table.schema().clone(); - let upstream_table_reader = UpstreamTableReader::new(self.upstream_table); + let upstream_table_id = self.external_table.table_id().table_id; + let upstream_table_schema = self.external_table.schema().clone(); + let upstream_table_reader = UpstreamTableReader::new(self.external_table); let mut upstream = self.upstream.execute(); @@ -133,11 +141,15 @@ impl CdcBackfillExecutor { // Check whether this parallelism has been assigned splits, // if not, we should bypass the backfill directly. let mut state_impl = if shared_cdc_source { - CdcBackfillStateImpl::Undefined + assert!(self.state_table.is_some(), "expect state table for shared cdc source"); + CdcBackfillStateImpl::MultiTable(MultiBackfillState::new(upstream_table_id, self.state_table.unwrap(), pk_in_output_indices.len() + METADATA_STATE_LEN)) } else if let Some(mutation) = first_barrier.mutation.as_ref() && let Mutation::Add{splits, ..} = mutation.as_ref() { tracing::info!(?mutation, ?shared_cdc_source, "got first barrier"); + + assert!(self.source_state_handler.is_some(), "expect source state handler"); + // We can assume for cdc table, the parallism of the fragment must be 1 match splits.get(&self.actor_ctx.id) { None => { @@ -164,7 +176,7 @@ impl CdcBackfillExecutor { upstream_table_id )) })?; - CdcBackfillStateImpl::SingleTable(SingleTableState::new(self.source_state_handler, upstream_table_id, split.id(), split.clone())) + CdcBackfillStateImpl::SingleTable(SingleBackfillState::new(self.source_state_handler.unwrap(), upstream_table_id, split.id(), split.clone())) } } } @@ -180,21 +192,16 @@ impl CdcBackfillExecutor { upstream.peekable() }; - tracing::debug!(?upstream_table_id, ?self.actor_ctx.id, ?shared_cdc_source, "start cdc backfill"); + tracing::debug!(?upstream_table_id, ?shared_cdc_source, "start cdc backfill"); state_impl.init_epoch(first_barrier.epoch); - // start from the beginning - // TODO(siyuan): restore backfill offset from state store - let backfill_offset = None; - - current_pk_pos = backfill_offset; - - // restore backfill done flag from state store - let is_finished = state_impl.check_finished().await?; + // restore backfill state + let state = state_impl.restore_state().await?; + current_pk_pos = state.current_pk_pos.clone(); // If the snapshot is empty, we don't need to backfill. let is_snapshot_empty: bool = { - if is_finished { + if state.is_finished { // It is finished, so just assign a value to avoid accessing storage table again. false } else { @@ -209,19 +216,18 @@ impl CdcBackfillExecutor { // | t | t/f | f | // | f | t | f | // | f | f | t | - let to_backfill = !is_finished && !is_snapshot_empty; + let to_backfill = !state.is_finished && !is_snapshot_empty; // The first barrier message should be propagated. yield Message::Barrier(first_barrier); // Keep track of rows from the snapshot. - #[allow(unused_variables)] - let mut total_snapshot_processed_rows: u64 = 0; + let mut total_snapshot_row_count: u64 = 0; let mut snapshot_read_epoch; - // Read the current binlog offset as a low watermark - let mut last_binlog_offset: Option = - upstream_table_reader.current_binlog_offset().await?; + let mut last_binlog_offset: Option = state + .last_cdc_offset + .map_or(upstream_table_reader.current_binlog_offset().await?, Some); let mut consumed_binlog_offset: Option = None; @@ -312,7 +318,7 @@ impl CdcBackfillExecutor { .backfill_snapshot_read_row_count .with_label_values(&[ upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), + self.actor_ctx.id.to_string().as_str(), ]) .inc_by(cur_barrier_snapshot_processed_rows); @@ -320,7 +326,7 @@ impl CdcBackfillExecutor { .backfill_upstream_output_row_count .with_label_values(&[ upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), + self.actor_ctx.id.to_string().as_str(), ]) .inc_by(cur_barrier_upstream_processed_rows); @@ -329,14 +335,23 @@ impl CdcBackfillExecutor { last_binlog_offset = consumed_binlog_offset.clone(); } - // seal current epoch even though there is no data + // update and persist backfill state + state_impl + .mutate_state( + current_pk_pos.clone(), + last_binlog_offset.clone(), + total_snapshot_row_count, + false, + ) + .await?; state_impl.commit_state(barrier.epoch).await?; + snapshot_read_epoch = barrier.epoch.prev; if let Some(progress) = self.progress.as_mut() { progress.update( barrier.epoch.curr, snapshot_read_epoch, - total_snapshot_processed_rows, + total_snapshot_row_count, ); } @@ -356,7 +371,8 @@ impl CdcBackfillExecutor { )?; tracing::trace!( - "recv changelog chunk: bin offset {:?}, capactiy {}", + target: "events::stream::cdc_backfill", + "recv changelog chunk: chunk_offset {:?}, capactiy {}", chunk_binlog_offset, chunk.capacity() ); @@ -364,16 +380,15 @@ impl CdcBackfillExecutor { // Since we don't need changelog before the // `last_binlog_offset`, skip the chunk that *only* contains // events before `last_binlog_offset`. - if let Some(last_binlog_offset) = &last_binlog_offset { - if let Some(chunk_binlog_offset) = chunk_binlog_offset { - if chunk_binlog_offset < *last_binlog_offset { - tracing::trace!( - "skip changelog chunk: offset {:?}, capacity {}", - chunk_binlog_offset, - chunk.capacity() - ); - continue; - } + if let Some(last_binlog_offset) = last_binlog_offset.as_ref() { + if let Some(chunk_offset) = chunk_binlog_offset && chunk_offset < *last_binlog_offset { + tracing::trace!( + target: "events::stream::cdc_backfill", + "skip changelog chunk: chunk_offset {:?}, capacity {}", + chunk_offset, + chunk.capacity() + ); + continue; } } // Buffer the upstream chunk. @@ -400,16 +415,20 @@ impl CdcBackfillExecutor { // in the buffer. Here we choose to never mark the chunk. // Consume with the renaming stream buffer chunk without mark. for chunk in upstream_chunk_buffer.drain(..) { - let chunk_cardinality = chunk.cardinality() as u64; - cur_barrier_snapshot_processed_rows += chunk_cardinality; - total_snapshot_processed_rows += chunk_cardinality; yield Message::Chunk(mapping_chunk( chunk, &self.output_indices, )); } - state_impl.mutate_state(last_binlog_offset.clone()).await?; + state_impl + .mutate_state( + current_pk_pos, + last_binlog_offset.clone(), + total_snapshot_row_count, + true, + ) + .await?; break 'backfill_loop; } Some(chunk) => { @@ -425,7 +444,7 @@ impl CdcBackfillExecutor { ); let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; - total_snapshot_processed_rows += chunk_cardinality; + total_snapshot_row_count += chunk_cardinality; yield Message::Chunk(mapping_chunk( chunk, &self.output_indices, @@ -443,14 +462,20 @@ impl CdcBackfillExecutor { "upstream snapshot is empty, mark backfill is done and persist current binlog offset"); // The snapshot is empty, just set backfill to finished - state_impl.mutate_state(last_binlog_offset).await?; + state_impl + .mutate_state( + current_pk_pos, + last_binlog_offset, + total_snapshot_row_count, + true, + ) + .await?; } // drop reader to release db connection drop(upstream_table_reader); tracing::info!( - actor = self.actor_id, "CdcBackfill has already finished and forward messages directly to the downstream" ); @@ -465,7 +490,7 @@ impl CdcBackfillExecutor { // mark progress as finished if let Some(progress) = self.progress.as_mut() { - progress.finish(barrier.epoch.curr, total_snapshot_processed_rows); + progress.finish(barrier.epoch.curr, total_snapshot_row_count); } yield msg; // break after the state have been saved diff --git a/src/stream/src/executor/backfill/cdc/mod.rs b/src/stream/src/executor/backfill/cdc/mod.rs index 5061211f28c6a..eae920fbab7e1 100644 --- a/src/stream/src/executor/backfill/cdc/mod.rs +++ b/src/stream/src/executor/backfill/cdc/mod.rs @@ -15,4 +15,4 @@ pub mod cdc_backfill; mod state; -pub use state::BACKFILL_STATE_KEY_SUFFIX; +pub use state::{CdcStateRecord, BACKFILL_STATE_KEY_SUFFIX}; diff --git a/src/stream/src/executor/backfill/cdc/state.rs b/src/stream/src/executor/backfill/cdc/state.rs index 2d32bbd1d4353..d211924c39ec8 100644 --- a/src/stream/src/executor/backfill/cdc/state.rs +++ b/src/stream/src/executor/backfill/cdc/state.rs @@ -12,61 +12,188 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::anyhow; use maplit::hashmap; -use risingwave_common::row::Row; -use risingwave_common::types::{JsonbVal, ScalarRefImpl}; +use risingwave_common::row; +use risingwave_common::row::{OwnedRow, Row}; +use risingwave_common::types::{Datum, JsonbVal, ScalarImpl, ScalarRefImpl}; use risingwave_common::util::epoch::EpochPair; use risingwave_connector::source::external::{CdcOffset, DebeziumOffset, DebeziumSourceOffset}; use risingwave_connector::source::{SplitId, SplitImpl, SplitMetaData}; use risingwave_storage::StateStore; use serde_json::Value; +use crate::common::table::state_table::StateTable; use crate::executor::{SourceStateTableHandler, StreamExecutorResult}; +/// Depending on how the table is created, we have two scenarios for CDC Backfill: +/// 1. `CREATE TABLE xx WITH ("connector"= 'mysql-cdc', "database.name"='mydb', "table.name"='t1')` +/// In this case, the cdc backfill executor will wraps the source executor, and maintain its state +/// (a finish flag) in the source state table. +/// +/// +/// 2. `CREATE TABLE xx FROM source TABLE 'mydb.t1'` +/// In this case, we can have multiple Table jobs sharing a single cdc Source job. +/// The cdc backfill executor will be an instance of the `StreamScan` operator and has its own state table +/// schema: `table_id | backfill_finished | row_count | cdc_offset` pub enum CdcBackfillStateImpl { - Undefined, - SingleTable(SingleTableState), + SingleTable(SingleBackfillState), + MultiTable(MultiBackfillState), +} + +#[derive(Debug, Default)] +pub struct CdcStateRecord { + pub current_pk_pos: Option, + pub is_finished: bool, + /// The last cdc offset that has been consumed by the cdc backfill executor + pub last_cdc_offset: Option, + pub row_count: i64, } impl CdcBackfillStateImpl { pub fn init_epoch(&mut self, epoch: EpochPair) { match self { - CdcBackfillStateImpl::Undefined => {} CdcBackfillStateImpl::SingleTable(state) => state.init_epoch(epoch), + CdcBackfillStateImpl::MultiTable(state) => state.init_epoch(epoch), } } - pub async fn check_finished(&self) -> StreamExecutorResult { + /// Restore the state of the corresponding split + pub async fn restore_state(&mut self) -> StreamExecutorResult { match self { - CdcBackfillStateImpl::Undefined => Ok(false), - CdcBackfillStateImpl::SingleTable(state) => state.check_finished().await, + CdcBackfillStateImpl::SingleTable(state) => state.restore_state().await, + CdcBackfillStateImpl::MultiTable(state) => state.restore_state().await, } } + /// Modify the state of the corresponding split (currently only supports single split) pub async fn mutate_state( &mut self, - last_binlog_offset: Option, + current_pk_pos: Option, + last_cdc_offset: Option, + row_count: u64, + is_finished: bool, ) -> StreamExecutorResult<()> { + let record = CdcStateRecord { + current_pk_pos, + last_cdc_offset, + row_count: row_count as _, + is_finished, + }; match self { - CdcBackfillStateImpl::Undefined => Ok(()), - CdcBackfillStateImpl::SingleTable(state) => { - state.mutate_state(last_binlog_offset).await - } + CdcBackfillStateImpl::SingleTable(state) => state.mutate_state(&record).await, + CdcBackfillStateImpl::MultiTable(state) => state.mutate_state(&record).await, } } + /// Persist the state to storage pub async fn commit_state(&mut self, new_epoch: EpochPair) -> StreamExecutorResult<()> { match self { - CdcBackfillStateImpl::Undefined => Ok(()), CdcBackfillStateImpl::SingleTable(state) => state.commit_state(new_epoch).await, + CdcBackfillStateImpl::MultiTable(state) => state.commit_state(new_epoch).await, } } } pub const BACKFILL_STATE_KEY_SUFFIX: &str = "_backfill"; -/// The state manager for single cdc table -pub struct SingleTableState { +pub struct MultiBackfillState { + /// Id of the backfilling table, will be the key of the state + split_id: String, + state_table: StateTable, + + cached_state: Vec, +} + +impl MultiBackfillState { + pub fn new(table_id: u32, state_table: StateTable, state_len: usize) -> Self { + Self { + split_id: table_id.to_string(), + state_table, + cached_state: vec![None; state_len], + } + } + + pub fn init_epoch(&mut self, epoch: EpochPair) { + self.state_table.init_epoch(epoch) + } + + /// Restore the backfill state from storage + pub async fn restore_state(&mut self) -> StreamExecutorResult { + let key = Some(self.split_id.clone()); + match self + .state_table + .get_row(row::once(key.map(ScalarImpl::from))) + .await? + { + Some(row) => { + self.cached_state = row.into_inner().into_vec(); + let state = self.cached_state.as_slice(); + let state_len = state.len(); + // schema: | `split_id` | `pk...` | `backfill_finished` | `row_count` | `cdc_offset` | + let cdc_offset = match state[state_len - 1] { + Some(ScalarImpl::Jsonb(ref jsonb)) => { + serde_json::from_value(jsonb.clone().take()).unwrap() + } + _ => return Err(anyhow!("invalid backfill state: cdc_offset").into()), + }; + let row_count = match state[state_len - 2] { + Some(ScalarImpl::Int64(val)) => val, + _ => return Err(anyhow!("invalid backfill state: row_count").into()), + }; + let is_finished = match state[state_len - 3] { + Some(ScalarImpl::Bool(val)) => val, + _ => return Err(anyhow!("invalid backfill state: backfill_finished").into()), + }; + + let current_pk_pos = state[1..state_len - 3].to_vec(); + Ok(CdcStateRecord { + current_pk_pos: Some(OwnedRow::new(current_pk_pos)), + is_finished, + last_cdc_offset: cdc_offset, + row_count, + }) + } + None => Ok(CdcStateRecord::default()), + } + } + + pub async fn mutate_state(&mut self, record: &CdcStateRecord) -> StreamExecutorResult<()> { + let Some(current_pk_pos) = &record.current_pk_pos else { + return Ok(()); + }; + + // schema: | `split_id` | `pk...` | `backfill_finished` | `row_count` | `cdc_offset` | + let state = self.cached_state.as_mut_slice(); + let split_id = Some(ScalarImpl::from(self.split_id.clone())); + state[0] = split_id.clone(); + state[1..=current_pk_pos.len()].clone_from_slice(current_pk_pos.as_inner()); + state[current_pk_pos.len() + 1] = Some(record.is_finished.into()); + state[current_pk_pos.len() + 2] = Some(record.row_count.into()); + state[current_pk_pos.len() + 3] = record.last_cdc_offset.clone().map(|cdc_offset| { + let json = serde_json::to_value(cdc_offset).unwrap(); + ScalarImpl::Jsonb(JsonbVal::from(json)) + }); + + match self.state_table.get_row(row::once(split_id)).await? { + Some(prev_row) => { + self.state_table + .update(prev_row, self.cached_state.as_slice()); + } + None => { + self.state_table.insert(self.cached_state.as_slice()); + } + } + Ok(()) + } + + pub async fn commit_state(&mut self, new_epoch: EpochPair) -> StreamExecutorResult<()> { + self.state_table.commit(new_epoch).await + } +} + +/// The state manager for backfilling a single table +pub struct SingleBackfillState { /// Stores the backfill done flag source_state_handler: SourceStateTableHandler, cdc_table_id: u32, @@ -74,9 +201,7 @@ pub struct SingleTableState { cdc_split: SplitImpl, } -impl SingleTableState {} - -impl SingleTableState { +impl SingleBackfillState { pub fn new( source_state_handler: SourceStateTableHandler, cdc_table_id: u32, @@ -95,37 +220,46 @@ impl SingleTableState { self.source_state_handler.init_epoch(epoch) } - pub async fn check_finished(&self) -> StreamExecutorResult { + pub async fn restore_state(&self) -> StreamExecutorResult { let mut key = self.split_id.to_string(); key.push_str(BACKFILL_STATE_KEY_SUFFIX); - match self.source_state_handler.get(key.into()).await? { + let is_finished = match self.source_state_handler.get(key.into()).await? { Some(row) => match row.datum_at(1) { - Some(ScalarRefImpl::Jsonb(jsonb_ref)) => Ok(jsonb_ref.as_bool()?), + Some(ScalarRefImpl::Jsonb(jsonb_ref)) => jsonb_ref.as_bool()?, _ => unreachable!("invalid backfill persistent state"), }, - None => Ok(false), - } + None => false, + }; + Ok(CdcStateRecord { + is_finished, + ..Default::default() + }) } - /// When snapshot read stream ends, we should persist two states: - /// 1) a backfill finish flag to denote the backfill has done - /// 2) a consumed binlog offset to denote the last binlog offset - /// which will be committed to the state store upon next barrier. - pub async fn mutate_state( - &mut self, - last_binlog_offset: Option, - ) -> StreamExecutorResult<()> { + pub async fn mutate_state(&mut self, state_item: &CdcStateRecord) -> StreamExecutorResult<()> { + // skip if unfinished for single backfill + if !state_item.is_finished { + return Ok(()); + } + + // When single backfill is finished, we should persist two states: + // 1) a finish flag to denote the backfill has done + // 2) a consumed binlog offset to denote the last binlog offset + // which will be committed to the state store upon next barrier. let mut key = self.split_id.to_string(); key.push_str(BACKFILL_STATE_KEY_SUFFIX); // write backfill finished flag self.source_state_handler - .set(key.into(), JsonbVal::from(Value::Bool(true))) + .set( + key.into(), + JsonbVal::from(Value::Bool(state_item.is_finished)), + ) .await?; if let SplitImpl::MysqlCdc(split) = &mut self.cdc_split && let Some(state) = split.mysql_split.as_mut() { let start_offset = - last_binlog_offset.as_ref().map(|cdc_offset| { + state_item.last_cdc_offset.as_ref().map(|cdc_offset| { let source_offset = if let CdcOffset::MySql(o) = cdc_offset { diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs index b6de1ac0daa4f..21b8ce35ed9ef 100644 --- a/src/stream/src/executor/dispatch.rs +++ b/src/stream/src/executor/dispatch.rs @@ -896,7 +896,7 @@ impl Dispatcher for CdcTableNameDispatcher { for vis_map in &mut vis_maps { let should_emit = if let Some(row) = row && let Some(full_table_name) = self.downstream_table_name.as_ref() { let table_name_datum = row.datum_at(self.table_name_col_index).unwrap(); - tracing::trace!(target: "events::stream::dispatch::hash::cdc", "keys: {:?}, table: {}", self.table_name_col_index, full_table_name); + tracing::trace!(target: "events::stream::dispatch::cdc", "keys: {:?}, table: {}", self.table_name_col_index, full_table_name); // dispatch based on downstream table name table_name_datum == ScalarRefImpl::Utf8(full_table_name) } else { diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 995a83b8f38fd..e78f3cd79c88e 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -183,7 +183,8 @@ impl ExecutorBuilder for SourceExecutorBuilder { (0..table_schema.len()).collect_vec(), None, params.executor_stats, - source_state_handler, + None, + Some(source_state_handler), false, source_ctrl_opts.chunk_size, ); diff --git a/src/stream/src/from_proto/stream_scan.rs b/src/stream/src/from_proto/stream_scan.rs index 0cb9b64bafbf5..5b6136fda9ea1 100644 --- a/src/stream/src/from_proto/stream_scan.rs +++ b/src/stream/src/from_proto/stream_scan.rs @@ -29,7 +29,7 @@ use crate::common::table::state_table::StateTable; use crate::executor::external::ExternalStorageTable; use crate::executor::{ BackfillExecutor, CdcBackfillExecutor, ChainExecutor, FlowControlExecutor, - RearrangedChainExecutor, SourceStateTableHandler, + RearrangedChainExecutor, }; pub struct StreamScanExecutorBuilder; @@ -102,11 +102,13 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { output_indices.clone(), ); - let source_state_handler = SourceStateTableHandler::from_table_catalog( - node.get_state_table().as_ref().unwrap(), - state_store.clone(), - ) - .await; + let vnodes = params.vnode_bitmap.map(Arc::new); + // cdc backfill should be singleton, so vnodes must be None. + assert_eq!(None, vnodes); + let state_table = + StateTable::from_table_catalog(node.get_state_table()?, state_store, vnodes) + .await; + CdcBackfillExecutor::new( params.actor_context.clone(), params.info, @@ -115,7 +117,8 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { output_indices, Some(progress), params.executor_stats, - source_state_handler, + Some(state_table), + None, true, params.env.config().developer.chunk_size, ) From cde91a0e4fd345e38938ce7a114341b21abc10b9 Mon Sep 17 00:00:00 2001 From: Croxx Date: Tue, 14 Nov 2023 15:55:09 +0800 Subject: [PATCH 67/77] feat: support foyer zstd compression (#13240) Signed-off-by: MrCroxx --- src/storage/src/store_impl.rs | 24 ++++++++++-------------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index 46afb0711cdae..734d559e826f7 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -557,13 +557,11 @@ impl StateStoreImpl { catalog_bits: opts.data_file_cache_catalog_bits, admissions: vec![], reinsertions: vec![], - compression: match opts.data_file_cache_compression.as_str() { - "none" => foyer::storage::compress::Compression::None, - _ => panic!( - "data file cache compression type not support: {}", - opts.data_file_cache_compression - ), - }, + compression: opts + .data_file_cache_compression + .as_str() + .try_into() + .map_err(HummockError::file_cache)?, }; let cache = FileCache::open(config) .await @@ -598,13 +596,11 @@ impl StateStoreImpl { catalog_bits: opts.meta_file_cache_catalog_bits, admissions: vec![], reinsertions: vec![], - compression: match opts.meta_file_cache_compression.as_str() { - "none" => foyer::storage::compress::Compression::None, - _ => panic!( - "meta file cache compression type not support: {}", - opts.meta_file_cache_compression - ), - }, + compression: opts + .meta_file_cache_compression + .as_str() + .try_into() + .map_err(HummockError::file_cache)?, }; FileCache::open(config) .await From ec3208b012a1e9cdb698b21f24b0e206efa56f31 Mon Sep 17 00:00:00 2001 From: Croxx Date: Tue, 14 Nov 2023 18:24:54 +0800 Subject: [PATCH 68/77] fix: panic if spawn when task drop with madsim (#13417) Signed-off-by: MrCroxx --- src/storage/src/hummock/sstable_store.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/storage/src/hummock/sstable_store.rs b/src/storage/src/hummock/sstable_store.rs index fe0e23e695845..f0b0accb897b3 100644 --- a/src/storage/src/hummock/sstable_store.rs +++ b/src/storage/src/hummock/sstable_store.rs @@ -106,6 +106,9 @@ impl LruCacheEventListener for BlockCacheEventListener { sst_id: key.0, block_idx: key.1, }; + // temporarily avoid spawn task while task drop with madsim + // FYI: https://github.com/madsim-rs/madsim/issues/182 + #[cfg(not(madsim))] self.data_file_cache .insert_if_not_exists_async(key, CachedBlock::Loaded { block: value }); } @@ -118,6 +121,9 @@ impl LruCacheEventListener for MetaCacheEventListener { type T = Box; fn on_release(&self, key: Self::K, value: Self::T) { + // temporarily avoid spawn task while task drop with madsim + // FYI: https://github.com/madsim-rs/madsim/issues/182 + #[cfg(not(madsim))] self.0.insert_if_not_exists_async(key, value); } } From 2df34ee858dae99227f4957585671cc09c736eb5 Mon Sep 17 00:00:00 2001 From: Dylan Date: Tue, 14 Nov 2023 18:35:44 +0800 Subject: [PATCH 69/77] fix(meta): fix ctrl c cancels streaming job (#13415) --- src/meta/src/manager/catalog/database.rs | 12 ++++++++++++ src/meta/src/manager/catalog/mod.rs | 14 +++++++++----- 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index 648f09951d6b2..d1a002408bc25 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -445,6 +445,18 @@ impl DatabaseManager { .map(|(k, _)| *k) } + pub fn find_persisted_creating_table_id(&self, key: &RelationKey) -> Option { + self.tables + .iter() + .find(|(_, t)| { + t.stream_job_status == PbStreamJobStatus::Creating as i32 + && t.database_id == key.0 + && t.schema_id == key.1 + && t.name == key.2 + }) + .map(|(k, _)| *k) + } + pub fn all_creating_streaming_jobs(&self) -> impl Iterator + '_ { self.in_progress_creation_streaming_job.keys().cloned() } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 9254702073e99..65daa69def417 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -2621,11 +2621,15 @@ impl CatalogManager { infos .into_iter() .flat_map(|info| { - guard.database.find_creating_streaming_job_id(&( - info.database_id, - info.schema_id, - info.name, - )) + let relation_key = &(info.database_id, info.schema_id, info.name); + guard + .database + .find_creating_streaming_job_id(relation_key) + .or_else(|| { + guard + .database + .find_persisted_creating_table_id(relation_key) + }) }) .collect_vec() } From 56b37c53e855a433d1aea9dce6c20a749b5d25c4 Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Wed, 15 Nov 2023 02:12:15 +0800 Subject: [PATCH 70/77] chore(deps): bump getrandom to v0.2.11 and its madsim patch (#13426) Signed-off-by: TennyZhuang Co-authored-by: xxchan --- Cargo.lock | 14 ++++---------- Cargo.toml | 2 +- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2ddc20b82e213..4ddae7dc53ff5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3582,9 +3582,8 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" +version = "0.2.11" +source = "git+https://github.com/madsim-rs/getrandom.git?rev=e79a7ae#e79a7aecbcf8a43a802d30742667b77d75c613bd" dependencies = [ "cfg-if", "js-sys", @@ -4473,9 +4472,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.148" +version = "0.2.150" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cdc71e17332e86d2e1d38c1f99edcb6288ee11b815fb1a4b049eaa2114d369b" +checksum = "89d92a4743f9a61002fae18374ed11e7973f530cb3a3255fb354818118b2203c" [[package]] name = "libflate" @@ -11677,8 +11676,3 @@ dependencies = [ "libc", "pkg-config", ] - -[[patch.unused]] -name = "getrandom" -version = "0.2.9" -source = "git+https://github.com/madsim-rs/getrandom.git?rev=8daf97e#8daf97e4142635fe28543b2db9022f5e2544bb5c" diff --git a/Cargo.toml b/Cargo.toml index 7820fa06cba81..dd6fcf7269853 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -241,7 +241,7 @@ debug = 1 [patch.crates-io] # Patch third-party crates for deterministic simulation. quanta = { git = "https://github.com/madsim-rs/quanta.git", rev = "948bdc3" } -getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "8daf97e" } +getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "e79a7ae" } tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "fe39bb8e" } tokio-retry = { git = "https://github.com/madsim-rs/rust-tokio-retry.git", rev = "95e2fd3" } tokio-postgres = { git = "https://github.com/madsim-rs/rust-postgres.git", rev = "ac00d88" } From 08606e8c80ae19286615273ea8ab02a6fd802e18 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 15 Nov 2023 10:14:28 +0800 Subject: [PATCH 71/77] chore(deps): Bump rust_decimal from 1.32.0 to 1.33.0 (#13428) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 84 +++++++++++--------------- src/tests/e2e_extended_mode/Cargo.toml | 2 +- 2 files changed, 37 insertions(+), 49 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4ddae7dc53ff5..6be570b2eabd9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1369,47 +1369,26 @@ dependencies = [ [[package]] name = "borsh" -version = "0.10.3" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4114279215a005bc675e386011e594e1d9b800918cea18fcadadcce864a2046b" +checksum = "bf617fabf5cdbdc92f774bfe5062d870f228b80056d41180797abf48bed4056e" dependencies = [ "borsh-derive", - "hashbrown 0.13.2", + "cfg_aliases", ] [[package]] name = "borsh-derive" -version = "0.10.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0754613691538d51f329cce9af41d7b7ca150bc973056f1156611489475f54f7" -dependencies = [ - "borsh-derive-internal", - "borsh-schema-derive-internal", - "proc-macro-crate 0.1.5", - "proc-macro2", - "syn 1.0.109", -] - -[[package]] -name = "borsh-derive-internal" -version = "0.10.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "afb438156919598d2c7bad7e1c0adf3d26ed3840dbc010db1a882a65583ca2fb" -dependencies = [ - "proc-macro2", - "quote", - "syn 1.0.109", -] - -[[package]] -name = "borsh-schema-derive-internal" -version = "0.10.3" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "634205cc43f74a1b9046ef87c4540ebda95696ec0f315024860cad7c5b0f5ccd" +checksum = "f404657a7ea7b5249e36808dff544bc88a28f26e0ac40009f674b7a009d14be3" dependencies = [ + "once_cell", + "proc-macro-crate 2.0.0", "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.37", + "syn_derive", ] [[package]] @@ -1625,6 +1604,12 @@ dependencies = [ "syn 2.0.37", ] +[[package]] +name = "cfg_aliases" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fd16c4719339c4530435d38e511904438d07cce7950afa3718a84ac36c10e89e" + [[package]] name = "chrono" version = "0.4.31" @@ -6403,21 +6388,21 @@ dependencies = [ [[package]] name = "proc-macro-crate" -version = "0.1.5" +version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d6ea3c4595b96363c13943497db34af4460fb474a95c43f4446ad341b8c9785" +checksum = "7f4c021e1093a56626774e81216a4ce732a735e5bad4868a03f3ed65ca0c3919" dependencies = [ - "toml 0.5.11", + "once_cell", + "toml_edit 0.19.15", ] [[package]] name = "proc-macro-crate" -version = "1.3.1" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f4c021e1093a56626774e81216a4ce732a735e5bad4868a03f3ed65ca0c3919" +checksum = "7e8366a6159044a37876a2b9817124296703c586a5c92e2c53751fa06d8d43e8" dependencies = [ - "once_cell", - "toml_edit 0.19.15", + "toml_edit 0.20.2", ] [[package]] @@ -8674,9 +8659,9 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.32.0" +version = "1.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4c4216490d5a413bc6d10fa4742bd7d4955941d062c0ef873141d6b0e7b30fd" +checksum = "076ba1058b036d3ca8bcafb1d54d0b0572e99d7ecd3e4222723e18ca8e9ca9a8" dependencies = [ "arrayvec", "borsh", @@ -10073,6 +10058,18 @@ dependencies = [ "unicode-ident", ] +[[package]] +name = "syn_derive" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1329189c02ff984e9736652b1631330da25eaa6bc639089ed4915d25446cbe7b" +dependencies = [ + "proc-macro-error", + "proc-macro2", + "quote", + "syn 2.0.37", +] + [[package]] name = "sync-point" version = "0.1.0" @@ -10470,15 +10467,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "toml" -version = "0.5.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4f7f0dd8d50a853a531c426359045b1998f04219d88799810762cd4ad314234" -dependencies = [ - "serde", -] - [[package]] name = "toml" version = "0.7.8" diff --git a/src/tests/e2e_extended_mode/Cargo.toml b/src/tests/e2e_extended_mode/Cargo.toml index 56012a64a25dc..56128c9b45f39 100644 --- a/src/tests/e2e_extended_mode/Cargo.toml +++ b/src/tests/e2e_extended_mode/Cargo.toml @@ -18,7 +18,7 @@ anyhow = { version = "1", features = ["backtrace"] } chrono = { version = "0.4", features = ['serde'] } clap = { version = "4", features = ["derive"] } pg_interval = "0.4" -rust_decimal ={ version = "1.32", features = ["db-postgres"] } +rust_decimal ={ version = "1.33", features = ["db-postgres"] } tokio = { version = "0.2.24", package = "madsim-tokio", features = ["rt", "macros","rt-multi-thread"] } tokio-postgres = { version = "0.7", features = ["with-chrono-0_4"] } tracing = "0.1" From 1a80ac031a8f0f8739bcdc014e29238b3348b090 Mon Sep 17 00:00:00 2001 From: August Date: Wed, 15 Nov 2023 10:31:50 +0800 Subject: [PATCH 72/77] feat: introduce fragment level functions in sql catalog controller (#13381) --- .../migration/src/m20230908_072257_init.rs | 40 +- src/meta/model_v2/src/actor.rs | 34 +- src/meta/model_v2/src/fragment.rs | 8 +- src/meta/model_v2/src/lib.rs | 16 +- src/meta/model_v2/src/prelude.rs | 1 + src/meta/model_v2/src/streaming_job.rs | 47 ++ src/meta/src/controller/catalog.rs | 43 +- src/meta/src/controller/fragment.rs | 602 ++++++++++++++++-- src/meta/src/controller/utils.rs | 40 +- 9 files changed, 761 insertions(+), 70 deletions(-) create mode 100644 src/meta/model_v2/src/streaming_job.rs diff --git a/src/meta/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model_v2/migration/src/m20230908_072257_init.rs index dc33c0c05b31f..13c7a7a4a1881 100644 --- a/src/meta/model_v2/migration/src/m20230908_072257_init.rs +++ b/src/meta/model_v2/migration/src/m20230908_072257_init.rs @@ -14,6 +14,7 @@ impl MigrationTrait for Migration { assert!(!manager.has_table(UserPrivilege::Table.to_string()).await?); assert!(!manager.has_table(Database::Table.to_string()).await?); assert!(!manager.has_table(Schema::Table.to_string()).await?); + assert!(!manager.has_table(StreamingJob::Table.to_string()).await?); assert!(!manager.has_table(Fragment::Table.to_string()).await?); assert!(!manager.has_table(Actor::Table.to_string()).await?); assert!(!manager.has_table(Table::Table.to_string()).await?); @@ -318,6 +319,25 @@ impl MigrationTrait for Migration { .to_owned(), ) .await?; + manager + .create_table( + MigrationTable::create() + .table(StreamingJob::Table) + .col(ColumnDef::new(StreamingJob::JobId).integer().primary_key()) + .col(ColumnDef::new(StreamingJob::JobStatus).string().not_null()) + .col(ColumnDef::new(StreamingJob::CreateType).string().not_null()) + .col(ColumnDef::new(StreamingJob::Timezone).string()) + .foreign_key( + &mut ForeignKey::create() + .name("FK_streaming_job_object_id") + .from(StreamingJob::Table, StreamingJob::JobId) + .to(Object::Table, Object::Oid) + .on_delete(ForeignKeyAction::Cascade) + .to_owned(), + ) + .to_owned(), + ) + .await?; manager .create_table( MigrationTable::create() @@ -328,7 +348,7 @@ impl MigrationTrait for Migration { .primary_key() .auto_increment(), ) - .col(ColumnDef::new(Fragment::TableId).integer().not_null()) + .col(ColumnDef::new(Fragment::JobId).integer().not_null()) .col( ColumnDef::new(Fragment::FragmentTypeMask) .integer() @@ -340,13 +360,13 @@ impl MigrationTrait for Migration { .not_null(), ) .col(ColumnDef::new(Fragment::StreamNode).json().not_null()) - .col(ColumnDef::new(Fragment::VnodeMapping).json()) + .col(ColumnDef::new(Fragment::VnodeMapping).json().not_null()) .col(ColumnDef::new(Fragment::StateTableIds).json()) .col(ColumnDef::new(Fragment::UpstreamFragmentId).json()) .foreign_key( &mut ForeignKey::create() .name("FK_fragment_table_id") - .from(Fragment::Table, Fragment::TableId) + .from(Fragment::Table, Fragment::JobId) .to(Object::Table, Object::Oid) .on_delete(ForeignKeyAction::Cascade) .to_owned(), @@ -365,7 +385,7 @@ impl MigrationTrait for Migration { .auto_increment(), ) .col(ColumnDef::new(Actor::FragmentId).integer().not_null()) - .col(ColumnDef::new(Actor::Status).json().not_null()) + .col(ColumnDef::new(Actor::Status).string().not_null()) .col(ColumnDef::new(Actor::Splits).json()) .col(ColumnDef::new(Actor::ParallelUnitId).integer().not_null()) .col(ColumnDef::new(Actor::UpstreamActorIds).json()) @@ -767,6 +787,7 @@ impl MigrationTrait for Migration { UserPrivilege, Database, Schema, + StreamingJob, Fragment, Actor, Table, @@ -854,7 +875,7 @@ enum Schema { enum Fragment { Table, FragmentId, - TableId, + JobId, FragmentTypeMask, DistributionType, StreamNode, @@ -876,6 +897,15 @@ enum Actor { VnodeBitmap, } +#[derive(DeriveIden)] +enum StreamingJob { + Table, + JobId, + JobStatus, + Timezone, + CreateType, +} + #[derive(DeriveIden)] #[allow(clippy::enum_variant_names)] enum Table { diff --git a/src/meta/model_v2/src/actor.rs b/src/meta/model_v2/src/actor.rs index dd3ed244209ea..75dd3806aca74 100644 --- a/src/meta/model_v2/src/actor.rs +++ b/src/meta/model_v2/src/actor.rs @@ -12,12 +12,38 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::meta::table_fragments::actor_status::PbActorState; use sea_orm::entity::prelude::*; -use crate::{ - ActorId, ActorStatus, ActorUpstreamActors, ConnectorSplits, Dispatchers, FragmentId, - VnodeBitmap, -}; +use crate::{ActorId, ActorUpstreamActors, ConnectorSplits, Dispatchers, FragmentId, VnodeBitmap}; + +#[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum)] +#[sea_orm(rs_type = "String", db_type = "String(None)")] +pub enum ActorStatus { + #[sea_orm(string_value = "INACTIVE")] + Inactive, + #[sea_orm(string_value = "RUNNING")] + Running, +} + +impl From for ActorStatus { + fn from(val: PbActorState) -> Self { + match val { + PbActorState::Unspecified => unreachable!(), + PbActorState::Inactive => ActorStatus::Inactive, + PbActorState::Running => ActorStatus::Running, + } + } +} + +impl From for PbActorState { + fn from(val: ActorStatus) -> Self { + match val { + ActorStatus::Inactive => PbActorState::Inactive, + ActorStatus::Running => PbActorState::Running, + } + } +} #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "actor")] diff --git a/src/meta/model_v2/src/fragment.rs b/src/meta/model_v2/src/fragment.rs index a364e0709e3d4..155995624fcc4 100644 --- a/src/meta/model_v2/src/fragment.rs +++ b/src/meta/model_v2/src/fragment.rs @@ -15,18 +15,18 @@ use risingwave_pb::meta::table_fragments::fragment::PbFragmentDistributionType; use sea_orm::entity::prelude::*; -use crate::{FragmentId, FragmentVnodeMapping, I32Array, StreamNode, TableId}; +use crate::{FragmentId, FragmentVnodeMapping, I32Array, ObjectId, StreamNode}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "fragment")] pub struct Model { #[sea_orm(primary_key)] pub fragment_id: FragmentId, - pub table_id: TableId, + pub job_id: ObjectId, pub fragment_type_mask: i32, pub distribution_type: DistributionType, pub stream_node: StreamNode, - pub vnode_mapping: Option, + pub vnode_mapping: FragmentVnodeMapping, pub state_table_ids: I32Array, pub upstream_fragment_id: I32Array, } @@ -65,7 +65,7 @@ pub enum Relation { Actor, #[sea_orm( belongs_to = "super::object::Entity", - from = "Column::TableId", + from = "Column::JobId", to = "super::object::Column::Oid", on_update = "NoAction", on_delete = "Cascade" diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index 1f101a29f2f77..04f41eb26fd30 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -15,6 +15,7 @@ use std::collections::{BTreeMap, HashMap}; use risingwave_pb::catalog::{PbCreateType, PbStreamJobStatus}; +use risingwave_pb::meta::table_fragments::PbState as PbStreamJobState; use sea_orm::{DeriveActiveEnum, EnumIter, FromJsonQueryResult}; use serde::{Deserialize, Serialize}; @@ -39,6 +40,7 @@ pub mod object_dependency; pub mod schema; pub mod sink; pub mod source; +pub mod streaming_job; pub mod system_parameter; pub mod table; pub mod user; @@ -92,6 +94,16 @@ impl From for PbStreamJobStatus { } } +// todo: deprecate job status in catalog and unify with this one. +impl From for PbStreamJobState { + fn from(status: JobStatus) -> Self { + match status { + JobStatus::Creating => PbStreamJobState::Creating, + JobStatus::Created => PbStreamJobState::Created, + } + } +} + #[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum)] #[sea_orm(rs_type = "String", db_type = "String(None)")] pub enum CreateType { @@ -187,10 +199,6 @@ derive_from_json_struct!(StreamNode, risingwave_pb::stream_plan::PbStreamNode); derive_from_json_struct!(Dispatchers, Vec); derive_from_json_struct!(ConnectorSplits, risingwave_pb::source::ConnectorSplits); -derive_from_json_struct!( - ActorStatus, - risingwave_pb::meta::table_fragments::PbActorStatus -); derive_from_json_struct!(VnodeBitmap, risingwave_pb::common::Buffer); derive_from_json_struct!( diff --git a/src/meta/model_v2/src/prelude.rs b/src/meta/model_v2/src/prelude.rs index ab9670f712f04..f55f6ebc9a49d 100644 --- a/src/meta/model_v2/src/prelude.rs +++ b/src/meta/model_v2/src/prelude.rs @@ -31,6 +31,7 @@ pub use super::object_dependency::Entity as ObjectDependency; pub use super::schema::Entity as Schema; pub use super::sink::Entity as Sink; pub use super::source::Entity as Source; +pub use super::streaming_job::Entity as StreamingJob; pub use super::system_parameter::Entity as SystemParameter; pub use super::table::Entity as Table; pub use super::user::Entity as User; diff --git a/src/meta/model_v2/src/streaming_job.rs b/src/meta/model_v2/src/streaming_job.rs new file mode 100644 index 0000000000000..e2fe673ebd057 --- /dev/null +++ b/src/meta/model_v2/src/streaming_job.rs @@ -0,0 +1,47 @@ +// 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 sea_orm::entity::prelude::*; + +use crate::{CreateType, JobStatus}; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +#[sea_orm(table_name = "streaming_job")] +pub struct Model { + #[sea_orm(primary_key, auto_increment = false)] + pub job_id: i32, + pub job_status: JobStatus, + pub create_type: CreateType, + pub timezone: Option, +} + +#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] +pub enum Relation { + #[sea_orm( + belongs_to = "super::object::Entity", + from = "Column::JobId", + to = "super::object::Column::Oid", + on_update = "NoAction", + on_delete = "Cascade" + )] + Object, +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::Object.def() + } +} + +impl ActiveModelBehavior for ActiveModel {} diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index fddd2bd44f88e..cd69cdb630f40 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::iter; +use std::sync::Arc; use anyhow::anyhow; use itertools::Itertools; @@ -53,6 +54,8 @@ use crate::manager::{MetaSrvEnv, NotificationVersion, StreamingJob}; use crate::rpc::ddl_controller::DropMode; use crate::{MetaError, MetaResult}; +pub type CatalogControllerRef = Arc; + /// `CatalogController` is the controller for catalog related operations, including database, schema, table, view, etc. pub struct CatalogController { pub(crate) env: MetaSrvEnv, @@ -61,9 +64,9 @@ pub struct CatalogController { #[derive(Clone, Default)] pub struct ReleaseContext { - streaming_jobs: Vec, - source_ids: Vec, - connections: Vec, + pub(crate) streaming_jobs: Vec, + pub(crate) source_ids: Vec, + pub(crate) connections: Vec, } impl CatalogController { @@ -978,6 +981,40 @@ impl CatalogController { Ok(version) } + + pub async fn alter_source_column( + &self, + pb_source: PbSource, + ) -> MetaResult { + let source_id = pb_source.id as SourceId; + let inner = self.inner.write().await; + let txn = inner.db.begin().await?; + + let original_version: i64 = Source::find_by_id(source_id) + .select_only() + .column(source::Column::Version) + .into_tuple() + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("source", source_id))?; + if original_version + 1 != pb_source.version as i64 { + return Err(MetaError::permission_denied( + "source version is stale".to_string(), + )); + } + + let source: source::ActiveModel = pb_source.clone().into(); + source.update(&txn).await?; + txn.commit().await?; + + let version = self + .notify_frontend_relation_info( + NotificationOperation::Update, + PbRelationInfo::Source(pb_source), + ) + .await; + Ok(version) + } } #[cfg(test)] diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 6fa5d8c2aac0f..306ef73345bde 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -12,26 +12,63 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, BTreeSet, HashMap}; +use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; use std::mem::swap; use anyhow::Context; +use itertools::Itertools; use risingwave_common::bail; use risingwave_common::util::stream_graph_visitor::visit_stream_node; +use risingwave_meta_model_v2::actor::ActorStatus; +use risingwave_meta_model_v2::prelude::{Actor, Fragment, StreamingJob}; use risingwave_meta_model_v2::{ - actor, fragment, ActorStatus, ConnectorSplits, Dispatchers, FragmentVnodeMapping, StreamNode, - TableId, VnodeBitmap, + actor, fragment, ActorId, ConnectorSplits, Dispatchers, FragmentId, FragmentVnodeMapping, + I32Array, ObjectId, StreamNode, TableId, VnodeBitmap, WorkerId, }; +use risingwave_pb::common::PbParallelUnit; +use risingwave_pb::ddl_service::PbTableJobType; +use risingwave_pb::meta::table_fragments::actor_status::PbActorState; use risingwave_pb::meta::table_fragments::fragment::PbFragmentDistributionType; use risingwave_pb::meta::table_fragments::{PbActorStatus, PbFragment, PbState}; -use risingwave_pb::meta::PbTableFragments; +use risingwave_pb::meta::{FragmentParallelUnitMapping, PbTableFragments}; use risingwave_pb::source::PbConnectorSplits; use risingwave_pb::stream_plan::stream_node::NodeBody; -use risingwave_pb::stream_plan::{PbStreamEnvironment, StreamActor}; +use risingwave_pb::stream_plan::{ + PbDispatchStrategy, PbFragmentTypeFlag, PbStreamActor, PbStreamEnvironment, +}; +use sea_orm::sea_query::{Expr, Value}; +use sea_orm::ActiveValue::Set; +use sea_orm::{ + ColumnTrait, EntityTrait, JoinType, ModelTrait, PaginatorTrait, QueryFilter, QuerySelect, + RelationTrait, TransactionTrait, +}; -use crate::controller::catalog::CatalogController; +use crate::controller::catalog::{CatalogController, CatalogControllerInner}; +use crate::controller::utils::get_parallel_unit_mapping; +use crate::manager::ActorInfos; +use crate::stream::SplitAssignment; use crate::MetaResult; +impl CatalogControllerInner { + /// List all fragment vnode mapping info + pub async fn all_running_fragment_mappings( + &self, + ) -> MetaResult + '_> { + let fragment_mappings: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() + .select_only() + .columns([fragment::Column::FragmentId, fragment::Column::VnodeMapping]) + .into_tuple() + .all(&self.db) + .await?; + Ok(fragment_mappings.into_iter().map(|(fragment_id, mapping)| { + FragmentParallelUnitMapping { + fragment_id: fragment_id as _, + mapping: Some(mapping.into_inner()), + } + })) + } +} + impl CatalogController { pub fn extract_fragment_and_actors_from_table_fragments( PbTableFragments { @@ -61,7 +98,7 @@ impl CatalogController { } pub fn extract_fragment_and_actors( - table_id: TableId, + job_id: ObjectId, pb_fragment: PbFragment, pb_actor_status: &HashMap, pb_actor_splits: &HashMap, @@ -112,7 +149,7 @@ impl CatalogController { } }); - let StreamActor { + let PbStreamActor { actor_id, fragment_id, nodes: _, @@ -123,9 +160,7 @@ impl CatalogController { } = actor; let splits = pb_actor_splits.get(&actor_id).cloned().map(ConnectorSplits); - let status = pb_actor_status.get(&actor_id).cloned().map(ActorStatus); - - let status = status.ok_or_else(|| { + let status = pb_actor_status.get(&actor_id).cloned().ok_or_else(|| { anyhow::anyhow!( "actor {} in fragment {} has no actor_status", actor_id, @@ -134,7 +169,6 @@ impl CatalogController { })?; let parallel_unit_id = status - .inner_ref() .parallel_unit .as_ref() .map(|parallel_unit| parallel_unit.id) @@ -156,7 +190,7 @@ impl CatalogController { actors.push(actor::Model { actor_id: actor_id as _, fragment_id: fragment_id as _, - status, + status: status.get_state().unwrap().into(), splits, parallel_unit_id, upstream_actor_ids: upstream_actors.into(), @@ -167,7 +201,7 @@ impl CatalogController { let upstream_fragment_id = pb_upstream_fragment_ids.into(); - let vnode_mapping = pb_vnode_mapping.map(FragmentVnodeMapping); + let vnode_mapping = pb_vnode_mapping.map(FragmentVnodeMapping).unwrap(); let stream_node = StreamNode(stream_node); @@ -177,7 +211,7 @@ impl CatalogController { let fragment = fragment::Model { fragment_id: pb_fragment_id as _, - table_id, + job_id, fragment_type_mask: pb_fragment_type_mask as _, distribution_type, stream_node, @@ -194,6 +228,7 @@ impl CatalogController { state: PbState, env: Option, fragments: Vec<(fragment::Model, Vec)>, + parallel_units_map: HashMap, ) -> MetaResult { let mut pb_fragments = HashMap::new(); let mut pb_actor_splits = HashMap::new(); @@ -201,7 +236,7 @@ impl CatalogController { for (fragment, actors) in fragments { let (fragment, fragment_actor_status, fragment_actor_splits) = - Self::compose_fragment(fragment, actors)?; + Self::compose_fragment(fragment, actors, ¶llel_units_map)?; pb_fragments.insert(fragment.fragment_id, fragment); @@ -225,6 +260,7 @@ impl CatalogController { pub(crate) fn compose_fragment( fragment: fragment::Model, actors: Vec, + parallel_units_map: &HashMap, ) -> MetaResult<( PbFragment, HashMap, @@ -232,7 +268,7 @@ impl CatalogController { )> { let fragment::Model { fragment_id, - table_id: _, + job_id: _, fragment_type_mask, distribution_type, stream_node, @@ -262,6 +298,7 @@ impl CatalogController { actor_id, fragment_id, status, + parallel_unit_id, splits, upstream_actor_ids, dispatchers, @@ -295,13 +332,24 @@ impl CatalogController { let pb_dispatcher = dispatchers.into_inner(); - pb_actor_status.insert(actor_id as _, status.into_inner()); + pb_actor_status.insert( + actor_id as _, + PbActorStatus { + parallel_unit: Some( + parallel_units_map + .get(&(parallel_unit_id as _)) + .unwrap() + .clone(), + ), + state: PbActorState::from(status) as _, + }, + ); if let Some(splits) = splits { pb_actor_splits.insert(actor_id as _, splits.into_inner()); } - pb_actors.push(StreamActor { + pb_actors.push(PbStreamActor { actor_id: actor_id as _, fragment_id: fragment_id as _, nodes: pb_nodes, @@ -313,7 +361,7 @@ impl CatalogController { } let pb_upstream_fragment_ids = upstream_fragment_id.into_u32_array(); - let pb_vnode_mapping = vnode_mapping.map(|mapping| mapping.into_inner()); + let pb_vnode_mapping = vnode_mapping.into_inner(); let pb_state_table_ids = state_table_ids.into_u32_array(); let pb_distribution_type = PbFragmentDistributionType::from(distribution_type) as _; let pb_fragment = PbFragment { @@ -321,13 +369,470 @@ impl CatalogController { fragment_type_mask: fragment_type_mask as _, distribution_type: pb_distribution_type, actors: pb_actors, - vnode_mapping: pb_vnode_mapping, + vnode_mapping: Some(pb_vnode_mapping), state_table_ids: pb_state_table_ids, upstream_fragment_ids: pb_upstream_fragment_ids, }; Ok((pb_fragment, pb_actor_status, pb_actor_splits)) } + + pub async fn running_fragment_parallelisms( + &self, + id_filter: Option>, + ) -> MetaResult> { + let inner = self.inner.read().await; + let mut select = Actor::find() + .select_only() + .column(actor::Column::FragmentId) + .column_as(actor::Column::ActorId.count(), "count") + .group_by(actor::Column::FragmentId); + if let Some(id_filter) = id_filter { + select = select.having(actor::Column::FragmentId.is_in(id_filter)); + } + let fragment_parallelisms: Vec<(FragmentId, i64)> = + select.into_tuple().all(&inner.db).await?; + Ok(fragment_parallelisms + .into_iter() + .map(|(fragment_id, count)| (fragment_id, count as usize)) + .collect()) + } + + pub async fn fragment_job_mapping(&self) -> MetaResult> { + let inner = self.inner.read().await; + let fragment_jobs: Vec<(FragmentId, ObjectId)> = Fragment::find() + .select_only() + .columns([fragment::Column::FragmentId, fragment::Column::JobId]) + .into_tuple() + .all(&inner.db) + .await?; + Ok(fragment_jobs.into_iter().collect()) + } + + /// Gets the counts for each upstream relation that each stream job + /// indicated by `table_ids` depends on. + /// For example in the following query: + /// ```sql + /// CREATE MATERIALIZED VIEW m1 AS + /// SELECT * FROM t1 JOIN t2 ON t1.a = t2.a JOIN t3 ON t2.b = t3.b + /// ``` + /// + /// We have t1 occurring once, and t2 occurring once. + pub async fn get_upstream_job_counts( + &self, + job_ids: Vec, + ) -> MetaResult>> { + let inner = self.inner.read().await; + let upstream_fragments: Vec<(ObjectId, i32, I32Array)> = Fragment::find() + .select_only() + .columns([ + fragment::Column::JobId, + fragment::Column::FragmentTypeMask, + fragment::Column::UpstreamFragmentId, + ]) + .filter(fragment::Column::JobId.is_in(job_ids)) + .into_tuple() + .all(&inner.db) + .await?; + + // filter out stream scan node. + let upstream_fragments = upstream_fragments + .into_iter() + .filter(|(_, mask, _)| (*mask & PbFragmentTypeFlag::StreamScan as i32) != 0) + .map(|(obj, _, upstream_fragments)| (obj, upstream_fragments.into_inner())) + .collect_vec(); + + // count by fragment id. + let upstream_fragment_counts = upstream_fragments + .iter() + .flat_map(|(_, upstream_fragments)| upstream_fragments.iter().cloned()) + .counts(); + + // get fragment id to job id mapping. + let fragment_job_ids: Vec<(FragmentId, ObjectId)> = Fragment::find() + .select_only() + .columns([fragment::Column::FragmentId, fragment::Column::JobId]) + .filter( + fragment::Column::FragmentId + .is_in(upstream_fragment_counts.keys().cloned().collect_vec()), + ) + .into_tuple() + .all(&inner.db) + .await?; + let fragment_job_mapping: HashMap = + fragment_job_ids.into_iter().collect(); + + // get upstream job counts. + let upstream_job_counts = upstream_fragments + .into_iter() + .map(|(job_id, upstream_fragments)| { + let upstream_job_counts = upstream_fragments + .into_iter() + .map(|upstream_fragment_id| { + let upstream_job_id = + fragment_job_mapping.get(&upstream_fragment_id).unwrap(); + ( + *upstream_job_id, + *upstream_fragment_counts.get(&upstream_fragment_id).unwrap(), + ) + }) + .collect(); + (job_id, upstream_job_counts) + }) + .collect(); + Ok(upstream_job_counts) + } + + pub async fn get_job_fragments_by_id(&self, job_id: ObjectId) -> MetaResult { + let inner = self.inner.read().await; + let fragment_actors = Fragment::find() + .find_with_related(Actor) + .filter(fragment::Column::JobId.eq(job_id)) + .all(&inner.db) + .await?; + let job_info = StreamingJob::find_by_id(job_id) + .one(&inner.db) + .await? + .ok_or_else(|| anyhow::anyhow!("job {} not found in database", job_id))?; + + let parallel_units_map = get_parallel_unit_mapping(&inner.db).await?; + Self::compose_table_fragments( + job_id as _, + job_info.job_status.into(), + job_info + .timezone + .map(|tz| PbStreamEnvironment { timezone: tz }), + fragment_actors, + parallel_units_map, + ) + } + + /// Get all actor ids in the target streaming jobs. + pub async fn get_job_actor_mapping( + &self, + job_ids: Vec, + ) -> MetaResult>> { + let inner = self.inner.read().await; + let job_actors: Vec<(ObjectId, ActorId)> = Actor::find() + .select_only() + .column(fragment::Column::JobId) + .column(actor::Column::ActorId) + .join(JoinType::InnerJoin, actor::Relation::Fragment.def()) + .filter(fragment::Column::JobId.is_in(job_ids)) + .into_tuple() + .all(&inner.db) + .await?; + Ok(job_actors.into_iter().into_group_map()) + } + + /// Try to get internal table ids of each streaming job, used by metrics collection. + pub async fn get_job_internal_table_ids(&self) -> Option)>> { + if let Ok(inner) = self.inner.try_read() { + if let Ok(job_state_tables) = Fragment::find() + .select_only() + .columns([fragment::Column::JobId, fragment::Column::StateTableIds]) + .into_tuple::<(ObjectId, I32Array)>() + .all(&inner.db) + .await + { + let mut job_internal_table_ids = HashMap::new(); + for (job_id, state_table_ids) in job_state_tables { + job_internal_table_ids + .entry(job_id) + .or_insert_with(Vec::new) + .extend(state_table_ids.into_inner()); + } + return Some(job_internal_table_ids.into_iter().collect()); + } + } + None + } + + pub async fn has_any_running_jobs(&self) -> MetaResult { + let inner = self.inner.read().await; + let count = Fragment::find().count(&inner.db).await?; + Ok(count > 0) + } + + pub fn table_fragments(&self) -> MetaResult> { + unimplemented!( + "This function is too heavy, we should avoid using it and implement others on demand." + ) + } + + /// Check if the fragment type mask is injectable. + fn is_injectable(fragment_type_mask: u32) -> bool { + (fragment_type_mask + & (PbFragmentTypeFlag::Source as u32 + | PbFragmentTypeFlag::Now as u32 + | PbFragmentTypeFlag::Values as u32 + | PbFragmentTypeFlag::BarrierRecv as u32)) + != 0 + } + + /// Used in [`crate::barrier::GlobalBarrierManager`], load all actor that need to be sent or + /// collected + pub async fn load_all_actor( + &self, + parallel_units_map: &HashMap, + check_state: impl Fn(PbActorState, ObjectId, ActorId) -> bool, + ) -> MetaResult { + let inner = self.inner.read().await; + let actor_info: Vec<(ActorId, ActorStatus, i32, ObjectId, i32)> = Actor::find() + .select_only() + .column(actor::Column::ActorId) + .column(actor::Column::Status) + .column(actor::Column::ParallelUnitId) + .column(fragment::Column::JobId) + .column(fragment::Column::FragmentTypeMask) + .join(JoinType::InnerJoin, actor::Relation::Fragment.def()) + .into_tuple() + .all(&inner.db) + .await?; + + let mut actor_maps = HashMap::new(); + let mut barrier_inject_actor_maps = HashMap::new(); + + for (actor_id, status, parallel_unit_id, job_id, type_mask) in actor_info { + let status = PbActorState::from(status); + let worker_id = parallel_units_map + .get(&(parallel_unit_id as _)) + .unwrap() + .worker_node_id; + if check_state(status, job_id, actor_id) { + actor_maps + .entry(worker_id) + .or_insert_with(Vec::new) + .push(actor_id as _); + if Self::is_injectable(type_mask as _) { + barrier_inject_actor_maps + .entry(worker_id) + .or_insert_with(Vec::new) + .push(actor_id as _); + } + } + } + + Ok(ActorInfos { + actor_maps, + barrier_inject_actor_maps, + }) + } + + pub async fn migrate_actors(&self, plan: HashMap) -> MetaResult<()> { + let inner = self.inner.read().await; + let txn = inner.db.begin().await?; + for (from_pu_id, to_pu_id) in plan { + Actor::update_many() + .col_expr( + actor::Column::ParallelUnitId, + Expr::value(Value::Int(Some(to_pu_id.id as _))), + ) + .filter(actor::Column::ParallelUnitId.eq(from_pu_id as i32)) + .exec(&txn) + .await?; + } + txn.commit().await?; + + Ok(()) + } + + pub async fn all_inuse_parallel_units(&self) -> MetaResult> { + let inner = self.inner.read().await; + let parallel_units: Vec = Actor::find() + .select_only() + .column(actor::Column::ParallelUnitId) + .distinct() + .into_tuple() + .all(&inner.db) + .await?; + Ok(parallel_units) + } + + pub async fn all_node_actors( + &self, + include_inactive: bool, + ) -> MetaResult>> { + let inner = self.inner.read().await; + let parallel_units_map = get_parallel_unit_mapping(&inner.db).await?; + let fragment_actors = if include_inactive { + Fragment::find() + .find_with_related(Actor) + .all(&inner.db) + .await? + } else { + Fragment::find() + .find_with_related(Actor) + .filter(actor::Column::Status.eq(ActorStatus::Running)) + .all(&inner.db) + .await? + }; + + let mut node_actors = HashMap::new(); + for (fragment, actors) in fragment_actors { + let (table_fragments, actor_status, _) = + Self::compose_fragment(fragment, actors, ¶llel_units_map)?; + for actor in table_fragments.actors { + let node_id = actor_status[&actor.actor_id] + .get_parallel_unit() + .unwrap() + .worker_node_id as WorkerId; + node_actors + .entry(node_id) + .or_insert_with(Vec::new) + .push(actor); + } + } + + Ok(node_actors) + } + + pub async fn update_actor_splits(&self, split_assignment: &SplitAssignment) -> MetaResult<()> { + let inner = self.inner.read().await; + let txn = inner.db.begin().await?; + for assignments in split_assignment.values() { + for (actor_id, splits) in assignments { + Actor::update(actor::ActiveModel { + actor_id: Set(*actor_id as _), + splits: Set(Some(ConnectorSplits(PbConnectorSplits { + splits: splits.iter().map(Into::into).collect(), + }))), + ..Default::default() + }) + .exec(&txn) + .await?; + } + } + txn.commit().await?; + + Ok(()) + } + + /// Get the actor ids of the fragment with `fragment_id` with `Running` status. + pub async fn get_running_actors_by_fragment( + &self, + fragment_id: FragmentId, + ) -> MetaResult> { + let inner = self.inner.read().await; + let actors: Vec = Actor::find() + .select_only() + .column(actor::Column::ActorId) + .filter(actor::Column::FragmentId.eq(fragment_id)) + .filter(actor::Column::Status.eq(ActorStatus::Running)) + .into_tuple() + .all(&inner.db) + .await?; + Ok(actors) + } + + pub async fn get_actors_by_job_ids(&self, job_ids: Vec) -> MetaResult> { + let inner = self.inner.read().await; + let actors: Vec = Actor::find() + .select_only() + .column(actor::Column::ActorId) + .join(JoinType::InnerJoin, actor::Relation::Fragment.def()) + .filter(fragment::Column::JobId.is_in(job_ids)) + .into_tuple() + .all(&inner.db) + .await?; + Ok(actors) + } + + /// Get and filter the upstream `Materialize` or `Source` fragments of the specified relations. + pub async fn get_upstream_root_fragments( + &self, + upstream_job_ids: Vec, + job_type: Option, + ) -> MetaResult> { + let inner = self.inner.read().await; + + let mut fragments = Fragment::find() + .filter(fragment::Column::JobId.is_in(upstream_job_ids)) + .all(&inner.db) + .await?; + fragments.retain(|f| match job_type { + Some(PbTableJobType::SharedCdcSource) => { + f.fragment_type_mask & PbFragmentTypeFlag::Source as i32 != 0 + } + // MV on MV, and other kinds of table job + None | Some(PbTableJobType::General) | Some(PbTableJobType::Unspecified) => { + f.fragment_type_mask & PbFragmentTypeFlag::Mview as i32 != 0 + } + }); + + let parallel_units_map = get_parallel_unit_mapping(&inner.db).await?; + let mut root_fragments = HashMap::new(); + for fragment in fragments { + let actors = fragment.find_related(Actor).all(&inner.db).await?; + root_fragments.insert( + fragment.job_id, + Self::compose_fragment(fragment, actors, ¶llel_units_map)?.0, + ); + } + + Ok(root_fragments) + } + + /// Get the downstream `Chain` fragments of the specified table. + pub async fn get_downstream_chain_fragments( + &self, + job_id: ObjectId, + ) -> MetaResult> { + let mview_fragment = self.get_mview_fragment(job_id).await?; + let downstream_dispatches: HashMap<_, _> = mview_fragment.actors[0] + .dispatcher + .iter() + .map(|d| { + let fragment_id = d.dispatcher_id as FragmentId; + let strategy = PbDispatchStrategy { + r#type: d.r#type, + dist_key_indices: d.dist_key_indices.clone(), + output_indices: d.output_indices.clone(), + downstream_table_name: d.downstream_table_name.clone(), + }; + (fragment_id, strategy) + }) + .collect(); + + let inner = self.inner.read().await; + let parallel_units_map = get_parallel_unit_mapping(&inner.db).await?; + let mut chain_fragments = vec![]; + for (fragment_id, dispatch_strategy) in downstream_dispatches { + let mut fragment_actors = Fragment::find_by_id(fragment_id) + .find_with_related(Actor) + .all(&inner.db) + .await?; + if fragment_actors.is_empty() { + bail!("No fragment found for fragment id {}", fragment_id); + } + assert_eq!(fragment_actors.len(), 1); + let (fragment, actors) = fragment_actors.pop().unwrap(); + let fragment = Self::compose_fragment(fragment, actors, ¶llel_units_map)?.0; + chain_fragments.push((dispatch_strategy, fragment)); + } + + Ok(chain_fragments) + } + + /// Get the `Materialize` fragment of the specified table. + pub async fn get_mview_fragment(&self, job_id: ObjectId) -> MetaResult { + let inner = self.inner.read().await; + let mut fragments = Fragment::find() + .filter(fragment::Column::JobId.eq(job_id)) + .all(&inner.db) + .await?; + fragments.retain(|f| f.fragment_type_mask & PbFragmentTypeFlag::Mview as i32 != 0); + if fragments.is_empty() { + bail!("No mview fragment found for job {}", job_id); + } + assert_eq!(fragments.len(), 1); + + let parallel_units_map = get_parallel_unit_mapping(&inner.db).await?; + let fragment = fragments.pop().unwrap(); + let actors = fragment.find_related(Actor).all(&inner.db).await?; + + Ok(Self::compose_fragment(fragment, actors, ¶llel_units_map)?.0) + } } #[cfg(test)] @@ -339,19 +844,21 @@ mod tests { use risingwave_common::hash::{ParallelUnitId, ParallelUnitMapping}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common::util::stream_graph_visitor::visit_stream_node; + use risingwave_meta_model_v2::actor::ActorStatus; use risingwave_meta_model_v2::fragment::DistributionType; use risingwave_meta_model_v2::{ - actor, fragment, ActorId, ActorStatus, ActorUpstreamActors, ConnectorSplits, Dispatchers, - FragmentId, FragmentVnodeMapping, I32Array, StreamNode, TableId, VnodeBitmap, + actor, fragment, ActorId, ActorUpstreamActors, ConnectorSplits, Dispatchers, FragmentId, + FragmentVnodeMapping, I32Array, ObjectId, StreamNode, TableId, VnodeBitmap, }; use risingwave_pb::common::ParallelUnit; + use risingwave_pb::meta::table_fragments::actor_status::PbActorState; use risingwave_pb::meta::table_fragments::fragment::PbFragmentDistributionType; use risingwave_pb::meta::table_fragments::{PbActorStatus, PbFragment}; use risingwave_pb::source::{PbConnectorSplit, PbConnectorSplits}; use risingwave_pb::stream_plan::stream_node::{NodeBody, PbNodeBody}; use risingwave_pb::stream_plan::{ Dispatcher, MergeNode, PbDispatcher, PbDispatcherType, PbFragmentTypeFlag, PbStreamActor, - PbStreamNode, PbUnionNode, StreamActor, + PbStreamNode, PbUnionNode, }; use crate::controller::catalog::CatalogController; @@ -361,7 +868,7 @@ mod tests { const TEST_UPSTREAM_FRAGMENT_ID: FragmentId = 2; - const TEST_TABLE_ID: TableId = 1; + const TEST_JOB_ID: ObjectId = 1; const TEST_STATE_TABLE_ID: TableId = 1000; @@ -476,7 +983,7 @@ mod tests { actor_id, PbActorStatus { parallel_unit: Some(parallel_units[actor_id as usize].clone()), - ..Default::default() + state: PbActorState::Running as _, }, ) }) @@ -485,7 +992,7 @@ mod tests { let pb_actor_splits = Default::default(); let (fragment, actors) = CatalogController::extract_fragment_and_actors( - TEST_TABLE_ID, + TEST_JOB_ID, pb_fragment.clone(), &pb_actor_status, &pb_actor_splits, @@ -543,6 +1050,11 @@ mod tests { }) .collect(); + let parallel_units_map = parallel_units + .iter() + .map(|parallel_unit| (parallel_unit.id, parallel_unit.clone())) + .collect(); + let actors = (0..actor_count) .map(|actor_id| { let parallel_unit_id = actor_id as ParallelUnitId; @@ -551,11 +1063,6 @@ mod tests { .remove(¶llel_unit_id) .map(|m| VnodeBitmap(m.to_protobuf())); - let actor_status = ActorStatus(PbActorStatus { - parallel_unit: Some(parallel_units[actor_id as usize].clone()), - ..Default::default() - }); - let actor_splits = Some(ConnectorSplits(PbConnectorSplits { splits: vec![PbConnectorSplit { split_type: "dummy".to_string(), @@ -570,7 +1077,7 @@ mod tests { actor::Model { actor_id: actor_id as ActorId, fragment_id: TEST_FRAGMENT_ID, - status: actor_status, + status: ActorStatus::Running, splits: actor_splits, parallel_unit_id: parallel_unit_id as i32, upstream_actor_ids: ActorUpstreamActors(actor_upstream_actor_ids), @@ -595,17 +1102,21 @@ mod tests { let fragment = fragment::Model { fragment_id: TEST_FRAGMENT_ID, - table_id: TEST_TABLE_ID, + job_id: TEST_JOB_ID, fragment_type_mask: 0, distribution_type: DistributionType::Hash, stream_node: StreamNode(stream_node), - vnode_mapping: Some(FragmentVnodeMapping(parallel_unit_mapping.to_protobuf())), + vnode_mapping: FragmentVnodeMapping(parallel_unit_mapping.to_protobuf()), state_table_ids: I32Array(vec![TEST_STATE_TABLE_ID]), upstream_fragment_id: I32Array::default(), }; - let (pb_fragment, pb_actor_status, pb_actor_splits) = - CatalogController::compose_fragment(fragment.clone(), actors.clone()).unwrap(); + let (pb_fragment, pb_actor_status, pb_actor_splits) = CatalogController::compose_fragment( + fragment.clone(), + actors.clone(), + ¶llel_units_map, + ) + .unwrap(); assert_eq!(pb_actor_status.len(), actor_count as usize); assert_eq!(pb_actor_splits.len(), actor_count as usize); @@ -629,8 +1140,8 @@ mod tests { fn check_actors( actors: Vec, - pb_actors: Vec, - pb_actor_status: HashMap, + pb_actors: Vec, + _pb_actor_status: HashMap, pb_actor_splits: HashMap, ) { for ( @@ -644,7 +1155,7 @@ mod tests { dispatchers, vnode_bitmap, }, - StreamActor { + PbStreamActor { actor_id: pb_actor_id, fragment_id: pb_fragment_id, nodes: pb_nodes, @@ -690,14 +1201,7 @@ mod tests { } }); - assert_eq!( - status, - pb_actor_status - .get(&pb_actor_id) - .cloned() - .map(ActorStatus) - .unwrap() - ); + assert_eq!(status, ActorStatus::Running); assert_eq!( splits, @@ -727,7 +1231,7 @@ mod tests { PbFragmentDistributionType::from(fragment.distribution_type) as i32 ); assert_eq!( - pb_vnode_mapping.map(FragmentVnodeMapping), + pb_vnode_mapping.map(FragmentVnodeMapping).unwrap(), fragment.vnode_mapping ); diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index 6291633ba81ca..eb61d9895b617 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -12,15 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; + use anyhow::anyhow; use risingwave_meta_model_migration::WithQuery; use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::{ connection, function, index, object, object_dependency, schema, sink, source, table, user, - user_privilege, view, DataTypeArray, DatabaseId, ObjectId, PrivilegeId, SchemaId, UserId, + user_privilege, view, worker_property, DataTypeArray, DatabaseId, I32Array, ObjectId, + PrivilegeId, SchemaId, UserId, WorkerId, }; use risingwave_pb::catalog::{PbConnection, PbFunction}; +use risingwave_pb::common::PbParallelUnit; use risingwave_pb::user::grant_privilege::{PbAction, PbActionWithGrantOption, PbObject}; use risingwave_pb::user::{PbGrantPrivilege, PbUserInfo}; use sea_orm::sea_query::{ @@ -557,3 +561,37 @@ pub fn extract_grant_obj_id(object: &PbObject) -> ObjectId { _ => unreachable!("invalid object type: {:?}", object), } } + +// todo: deprecate parallel units and avoid this query. +pub async fn get_parallel_unit_mapping(db: &C) -> MetaResult> +where + C: ConnectionTrait, +{ + let parallel_units: Vec<(WorkerId, I32Array)> = WorkerProperty::find() + .select_only() + .columns([ + worker_property::Column::WorkerId, + worker_property::Column::ParallelUnitIds, + ]) + .into_tuple() + .all(db) + .await?; + let parallel_units_map = parallel_units + .into_iter() + .flat_map(|(worker_id, parallel_unit_ids)| { + parallel_unit_ids + .into_inner() + .into_iter() + .map(move |parallel_unit_id| { + ( + parallel_unit_id as _, + PbParallelUnit { + id: parallel_unit_id as _, + worker_node_id: worker_id as _, + }, + ) + }) + }) + .collect(); + Ok(parallel_units_map) +} From 190b46e294d627de4b175655516e829221f992a1 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed, 15 Nov 2023 10:32:45 +0800 Subject: [PATCH 73/77] fix(frontend): refine error message on missing `force_append_only` (#13413) --- src/frontend/src/optimizer/plan_node/stream_sink.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index cd48fda60e06b..5dbc7b8af62f4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -238,14 +238,16 @@ impl StreamSink { format_desc: Option<&SinkFormatDesc>, ) -> Result { let frontend_derived_append_only = input_append_only; - let (user_defined_append_only, user_force_append_only) = match format_desc { + let (user_defined_append_only, user_force_append_only, syntax_legacy) = match format_desc { Some(f) => ( f.format == SinkFormat::AppendOnly, Self::is_user_force_append_only(&WithOptions::from_inner(f.options.clone()))?, + false, ), None => ( Self::is_user_defined_append_only(properties)?, Self::is_user_force_append_only(properties)?, + true, ), }; @@ -260,14 +262,14 @@ impl StreamSink { (false, true, false) => { Err(ErrorCode::SinkError(Box::new(Error::new( ErrorKind::InvalidInput, - "The sink cannot be append-only. Please add \"force_append_only='true'\" in options to force the sink to be append-only. Notice that this will cause the sink executor to drop any UPDATE or DELETE message.", + format!("The sink cannot be append-only. Please add \"force_append_only='true'\" in {} options to force the sink to be append-only. Notice that this will cause the sink executor to drop any UPDATE or DELETE message.", if syntax_legacy {"WITH"} else {"FORMAT ENCODE"}), ))) .into()) } (_, false, true) => { Err(ErrorCode::SinkError(Box::new(Error::new( ErrorKind::InvalidInput, - "Cannot force the sink to be append-only without \"FORMAT PLAIN\" or \"type='append-only'\".", + format!("Cannot force the sink to be append-only without \"{}\".", if syntax_legacy {"type='append-only'"} else {"FORMAT PLAIN"}), ))) .into()) } From 5c54fe6d3be339f455cb42f50308ea363a95c8e6 Mon Sep 17 00:00:00 2001 From: Wallace Date: Wed, 15 Nov 2023 11:10:07 +0800 Subject: [PATCH 74/77] feat(storage): prefetch large data for backfill (#13132) Signed-off-by: Little-Wallace --- .../executor/join/distributed_lookup_join.rs | 2 +- src/batch/src/executor/row_seq_scan.rs | 2 +- src/common/src/cache.rs | 4 +- src/common/src/config.rs | 15 + src/compute/src/memory_management/mod.rs | 8 +- src/config/example.toml | 1 + src/ctl/src/cmd_impl/bench.rs | 6 +- src/ctl/src/cmd_impl/hummock/list_kv.rs | 2 +- src/ctl/src/cmd_impl/table/scan.rs | 2 +- src/ctl/src/common/hummock_service.rs | 1 + src/jni_core/src/hummock_iterator.rs | 8 +- src/object_store/src/object/mem.rs | 42 ++- src/object_store/src/object/mod.rs | 73 +++-- .../opendal_engine/opendal_object_store.rs | 29 +- src/object_store/src/object/s3.rs | 28 +- src/storage/benches/bench_compactor.rs | 2 + src/storage/benches/bench_multi_builder.rs | 1 + .../benches/bench_hummock_iter.rs | 2 +- .../hummock_test/src/bin/replay/main.rs | 1 + .../hummock_test/src/compactor_tests.rs | 6 +- .../hummock_test/src/failpoint_tests.rs | 2 +- .../hummock_test/src/hummock_storage_tests.rs | 30 +- .../hummock_test/src/snapshot_tests.rs | 2 +- .../hummock_test/src/state_store_tests.rs | 8 +- .../src/hummock/iterator/backward_concat.rs | 38 +-- .../src/hummock/iterator/backward_merge.rs | 94 +----- .../src/hummock/iterator/backward_user.rs | 152 ++------- .../src/hummock/iterator/forward_concat.rs | 46 +-- .../src/hummock/iterator/forward_merge.rs | 162 ++++------ .../src/hummock/iterator/forward_user.rs | 167 ++-------- .../src/hummock/iterator/test_utils.rs | 45 ++- .../sstable/backward_sstable_iterator.rs | 16 +- .../sstable/delete_range_aggregator.rs | 19 +- .../sstable/forward_sstable_iterator.rs | 298 +++++++----------- src/storage/src/hummock/sstable/mod.rs | 22 +- src/storage/src/hummock/sstable_store.rs | 270 ++++++++++++++-- .../src/hummock/store/hummock_storage.rs | 1 + src/storage/src/hummock/store/version.rs | 19 +- src/storage/src/hummock/test_utils.rs | 25 +- src/storage/src/hummock/validator.rs | 1 + src/storage/src/opts.rs | 5 + .../src/storage_failpoints/test_iterator.rs | 68 +--- .../src/storage_failpoints/test_sstable.rs | 8 +- src/storage/src/store.rs | 14 +- src/storage/src/store_impl.rs | 1 + .../log_store_impl/kv_log_store/reader.rs | 5 +- .../src/common/table/test_state_table.rs | 30 +- src/stream/src/executor/aggregation/minput.rs | 2 +- .../executor/backfill/no_shuffle_backfill.rs | 2 +- src/stream/src/executor/batch_query.rs | 2 +- src/stream/src/executor/dynamic_filter.rs | 2 +- src/stream/src/executor/lookup/impl_.rs | 4 +- .../src/executor/managed_state/join/mod.rs | 13 +- src/stream/src/executor/over_window/eowc.rs | 6 +- .../executor/over_window/over_partition.rs | 6 +- .../src/executor/source/fetch_executor.rs | 2 +- .../executor/source/state_table_handler.rs | 6 +- src/stream/src/executor/temporal_join.rs | 2 +- src/stream/src/executor/top_n/top_n_state.rs | 4 +- .../src/delete_range_runner.rs | 5 +- 60 files changed, 829 insertions(+), 1010 deletions(-) diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index 42b02df850163..6f48170107482 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -407,7 +407,7 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder { &pk_prefix, .., false, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::default(), ) .await?; diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index 1a94e9ef25b4c..55ade04c32996 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -416,7 +416,7 @@ impl RowSeqScanExecutor { end_bound.map(|x| OwnedRow::new(vec![x])), ), ordered, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::new_for_large_range_scan(), ) .await?; diff --git a/src/common/src/cache.rs b/src/common/src/cache.rs index f6af1ec60c0da..a3b20085f99f3 100644 --- a/src/common/src/cache.rs +++ b/src/common/src/cache.rs @@ -708,9 +708,9 @@ impl LruCache { } pub fn contains(self: &Arc, hash: u64, key: &K) -> bool { - let mut shard = self.shards[self.shard(hash)].lock(); + let shard = self.shards[self.shard(hash)].lock(); unsafe { - let ptr = shard.lookup(hash, key); + let ptr = shard.table.lookup(hash, key); !ptr.is_null() } } diff --git a/src/common/src/config.rs b/src/common/src/config.rs index ddb4ed8ec1c47..0090612d20ea3 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -503,6 +503,10 @@ pub struct StorageConfig { #[serde(default)] pub meta_cache_capacity_mb: Option, + /// max memory usage for large query + #[serde(default)] + pub large_query_memory_usage_mb: Option, + #[serde(default = "default::storage::disable_remote_compactor")] pub disable_remote_compactor: bool, @@ -587,6 +591,8 @@ pub struct StorageConfig { pub compactor_max_sst_size: u64, #[serde(default = "default::storage::enable_fast_compaction")] pub enable_fast_compaction: bool, + #[serde(default = "default::storage::max_preload_io_retry_times")] + pub max_preload_io_retry_times: usize, #[serde(default, flatten)] pub unrecognized: Unrecognized, @@ -1106,6 +1112,9 @@ pub mod default { true } + pub fn max_preload_io_retry_times() -> usize { + 3 + } pub fn mem_table_spill_threshold() -> usize { 4 << 20 } @@ -1406,6 +1415,7 @@ pub struct StorageMemoryConfig { pub data_file_cache_ring_buffer_capacity_mb: usize, pub meta_file_cache_ring_buffer_capacity_mb: usize, pub compactor_memory_limit_mb: usize, + pub large_query_memory_usage_mb: usize, pub high_priority_ratio_in_percent: usize, } @@ -1432,6 +1442,10 @@ pub fn extract_storage_memory_config(s: &RwConfig) -> StorageMemoryConfig { .storage .high_priority_ratio_in_percent .unwrap_or(default::storage::high_priority_ratio_in_percent()); + let large_query_memory_usage_mb = s + .storage + .shared_buffer_capacity_mb + .unwrap_or((100 - high_priority_ratio_in_percent) * block_cache_capacity_mb / 100); StorageMemoryConfig { block_cache_capacity_mb, @@ -1440,6 +1454,7 @@ pub fn extract_storage_memory_config(s: &RwConfig) -> StorageMemoryConfig { data_file_cache_ring_buffer_capacity_mb, meta_file_cache_ring_buffer_capacity_mb, compactor_memory_limit_mb, + large_query_memory_usage_mb, high_priority_ratio_in_percent, } } diff --git a/src/compute/src/memory_management/mod.rs b/src/compute/src/memory_management/mod.rs index aed720e36786f..a7e9baa15b369 100644 --- a/src/compute/src/memory_management/mod.rs +++ b/src/compute/src/memory_management/mod.rs @@ -42,7 +42,9 @@ pub const STORAGE_BLOCK_CACHE_MEMORY_PROPORTION: f64 = 0.3; pub const STORAGE_META_CACHE_MAX_MEMORY_MB: usize = 4096; pub const STORAGE_META_CACHE_MEMORY_PROPORTION: f64 = 0.35; pub const STORAGE_SHARED_BUFFER_MEMORY_PROPORTION: f64 = 0.3; -pub const STORAGE_DEFAULT_HIGH_PRIORITY_BLOCK_CACHE_RATIO: usize = 70; +pub const STORAGE_DEFAULT_HIGH_PRIORITY_BLOCK_CACHE_RATIO: usize = 50; +// Since the new feature prefetch does not cost much memory, we set a large value by default for performance. If we meet OOM during long time batch query, we shall reduce this configuration. +pub const STORAGE_DEFAULT_LARGE_QUERY_MEMORY_USAGE_MB: usize = 32 * 1024; /// `MemoryControlStats` contains the state from previous control loop #[derive(Default)] @@ -144,6 +146,9 @@ pub fn storage_memory_config( default_meta_cache_capacity >> 20, STORAGE_META_CACHE_MAX_MEMORY_MB, )); + let large_query_memory_usage_mb = storage_config + .large_query_memory_usage_mb + .unwrap_or(STORAGE_DEFAULT_LARGE_QUERY_MEMORY_USAGE_MB); if meta_cache_capacity_mb == STORAGE_META_CACHE_MAX_MEMORY_MB { block_cache_capacity_mb += (default_meta_cache_capacity >> 20) - meta_cache_capacity_mb; } @@ -196,6 +201,7 @@ pub fn storage_memory_config( data_file_cache_ring_buffer_capacity_mb, meta_file_cache_ring_buffer_capacity_mb, compactor_memory_limit_mb, + large_query_memory_usage_mb, high_priority_ratio_in_percent, } } diff --git a/src/config/example.toml b/src/config/example.toml index c5c9bb3aef021..301386d2ddb90 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -113,6 +113,7 @@ compactor_max_sst_key_count = 2097152 compact_iter_recreate_timeout_ms = 600000 compactor_max_sst_size = 536870912 enable_fast_compaction = true +max_preload_io_retry_times = 3 mem_table_spill_threshold = 4194304 [storage.data_file_cache] diff --git a/src/ctl/src/cmd_impl/bench.rs b/src/ctl/src/cmd_impl/bench.rs index 7dfd798a2b5be..32dacaa673a95 100644 --- a/src/ctl/src/cmd_impl/bench.rs +++ b/src/ctl/src/cmd_impl/bench.rs @@ -108,11 +108,7 @@ pub async fn do_bench(context: &CtlContext, cmd: BenchCommands) -> Result<()> { let sub_range: &(Bound, Bound) = &(Unbounded, Unbounded); let stream = state_table - .iter_with_prefix( - row::empty(), - sub_range, - PrefetchOptions::new_for_exhaust_iter(), - ) + .iter_with_prefix(row::empty(), sub_range, PrefetchOptions::default()) .await?; pin_mut!(stream); iter_cnt.fetch_add(1, std::sync::atomic::Ordering::Relaxed); diff --git a/src/ctl/src/cmd_impl/hummock/list_kv.rs b/src/ctl/src/cmd_impl/hummock/list_kv.rs index 15e59b23a2451..f336a4e3bcbcb 100644 --- a/src/ctl/src/cmd_impl/hummock/list_kv.rs +++ b/src/ctl/src/cmd_impl/hummock/list_kv.rs @@ -43,7 +43,7 @@ pub async fn list_kv( None, ReadOptions { table_id: TableId { table_id }, - prefetch_options: PrefetchOptions::new_for_exhaust_iter(), + prefetch_options: PrefetchOptions::default(), cache_policy: CachePolicy::NotFill, ..Default::default() }, diff --git a/src/ctl/src/cmd_impl/table/scan.rs b/src/ctl/src/cmd_impl/table/scan.rs index 4c1ff68963613..d7a737fe7d195 100644 --- a/src/ctl/src/cmd_impl/table/scan.rs +++ b/src/ctl/src/cmd_impl/table/scan.rs @@ -124,7 +124,7 @@ async fn do_scan(table: TableCatalog, hummock: MonitoredStateStore, - ) -> ObjectResult> { + read_range: Range, + ) -> ObjectResult { fail_point!("mem_streaming_read_err", |_| Err(ObjectError::internal( "mem streaming read error" ))); - let bytes = self - .get_object(path, start_pos.unwrap_or_default()..) - .await?; - Ok(Box::new(Cursor::new(bytes))) + let bytes = self.get_object(path, read_range).await?; + + Ok(Box::pin(InMemDataIterator::new(bytes))) } async fn metadata(&self, path: &str) -> ObjectResult { @@ -206,6 +204,32 @@ impl ObjectStore for InMemObjectStore { } } +pub struct InMemDataIterator { + data: Bytes, + offset: usize, +} + +impl InMemDataIterator { + pub fn new(data: Bytes) -> Self { + Self { data, offset: 0 } + } +} + +impl Stream for InMemDataIterator { + type Item = ObjectResult; + + fn poll_next(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { + const MAX_PACKET_SIZE: usize = 128 * 1024; + if self.offset >= self.data.len() { + return Poll::Ready(None); + } + let read_len = std::cmp::min(self.data.len() - self.offset, MAX_PACKET_SIZE); + let data = self.data.slice(self.offset..(self.offset + read_len)); + self.offset += read_len; + Poll::Ready(Some(Ok(data))) + } +} + static SHARED: LazyLock> = LazyLock::new(|| spin::Mutex::new(InMemObjectStore::new())); diff --git a/src/object_store/src/object/mod.rs b/src/object_store/src/object/mod.rs index fe80794756246..72a94ad626c12 100644 --- a/src/object_store/src/object/mod.rs +++ b/src/object_store/src/object/mod.rs @@ -12,13 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::ops::RangeBounds; +use std::ops::{Range, RangeBounds}; use std::sync::Arc; use std::time::Duration; use bytes::Bytes; use prometheus::HistogramTimer; -use tokio::io::{AsyncRead, AsyncReadExt}; pub mod mem; pub use mem::*; @@ -29,6 +28,7 @@ pub use opendal_engine::*; pub mod s3; use await_tree::InstrumentAwait; use futures::stream::BoxStream; +use futures::StreamExt; pub use s3::*; pub mod error; @@ -98,8 +98,8 @@ pub trait ObjectStore: Send + Sync { async fn streaming_read( &self, path: &str, - start_pos: Option, - ) -> ObjectResult>; + read_range: Range, + ) -> ObjectResult; /// Obtains the object metadata. async fn metadata(&self, path: &str) -> ObjectResult; @@ -121,6 +121,11 @@ pub trait ObjectStore: Send + Sync { async fn list(&self, prefix: &str) -> ObjectResult; fn store_media_type(&self) -> &'static str; + + fn recv_buffer_size(&self) -> usize { + // 2MB + 1 << 21 + } } pub enum ObjectStoreImpl { @@ -205,7 +210,7 @@ impl ObjectStoreImpl { pub async fn streaming_read( &self, path: &str, - start_loc: Option, + start_loc: Range, ) -> ObjectResult { object_store_impl_method_body!(self, streaming_read, dispatch_async, path, start_loc) } @@ -282,6 +287,14 @@ impl ObjectStoreImpl { } } } + + pub fn recv_buffer_size(&self) -> usize { + match self { + ObjectStoreImpl::InMem(store) => store.recv_buffer_size(), + ObjectStoreImpl::Opendal(store) => store.recv_buffer_size(), + ObjectStoreImpl::S3(store) => store.recv_buffer_size(), + } + } } fn try_update_failure_metric( @@ -407,9 +420,8 @@ impl MonitoredStreamingUploader { } } -type BoxedStreamingReader = Box; pub struct MonitoredStreamingReader { - inner: BoxedStreamingReader, + inner: ObjectDataStream, object_store_metrics: Arc, operation_size: usize, media_type: &'static str, @@ -417,10 +429,12 @@ pub struct MonitoredStreamingReader { streaming_read_timeout: Option, } +unsafe impl Sync for MonitoredStreamingReader {} + impl MonitoredStreamingReader { pub fn new( media_type: &'static str, - handle: BoxedStreamingReader, + handle: ObjectDataStream, object_store_metrics: Arc, streaming_read_timeout: Option, ) -> Self { @@ -439,42 +453,42 @@ impl MonitoredStreamingReader { } } - // This is a clippy bug, see https://github.com/rust-lang/rust-clippy/issues/11380. - // TODO: remove `allow` here after the issued is closed. - #[expect(clippy::needless_pass_by_ref_mut)] - pub async fn read_bytes(&mut self, buf: &mut [u8]) -> ObjectResult { + pub async fn read_bytes(&mut self) -> Option> { let operation_type = "streaming_read_read_bytes"; - let data_len = buf.len(); - self.object_store_metrics.read_bytes.inc_by(data_len as u64); - self.object_store_metrics - .operation_size - .with_label_values(&[operation_type]) - .observe(data_len as f64); let _timer = self .object_store_metrics .operation_latency .with_label_values(&[self.media_type, operation_type]) .start_timer(); - self.operation_size += data_len; let future = async { self.inner - .read_exact(buf) + .next() .verbose_instrument_await("object_store_streaming_read_read_bytes") .await - .map_err(|err| { - ObjectError::internal(format!("read_bytes failed, error: {:?}", err)) - }) }; let res = match self.streaming_read_timeout.as_ref() { None => future.await, Some(timeout) => tokio::time::timeout(*timeout, future) .await .unwrap_or_else(|_| { - Err(ObjectError::internal("streaming_read read_bytes timeout")) + Some(Err(ObjectError::internal( + "streaming_read read_bytes timeout", + ))) }), }; - try_update_failure_metric(&self.object_store_metrics, &res, operation_type); + if let Some(ret) = &res { + try_update_failure_metric(&self.object_store_metrics, ret, operation_type); + } + if let Some(Ok(data)) = &res { + let data_len = data.len(); + self.object_store_metrics.read_bytes.inc_by(data_len as u64); + self.object_store_metrics + .operation_size + .with_label_values(&[operation_type]) + .observe(data_len as f64); + self.operation_size += data_len; + } res } } @@ -635,7 +649,7 @@ impl MonitoredObjectStore { async fn streaming_read( &self, path: &str, - start_pos: Option, + range: Range, ) -> ObjectResult { let operation_type = "streaming_read_start"; let media_type = self.media_type(); @@ -646,7 +660,7 @@ impl MonitoredObjectStore { .start_timer(); let future = async { self.inner - .streaming_read(path, start_pos) + .streaming_read(path, range) .verbose_instrument_await("object_store_streaming_read") .await }; @@ -778,6 +792,10 @@ impl MonitoredObjectStore { self.read_timeout = Some(Duration::from_millis(read_timeout_ms)); self.upload_timeout = Some(Duration::from_millis(upload_timeout_ms)); } + + fn recv_buffer_size(&self) -> usize { + self.inner.recv_buffer_size() + } } pub async fn parse_remote_object_store( @@ -885,3 +903,4 @@ pub async fn parse_remote_object_store( } pub type ObjectMetadataIter = BoxStream<'static, ObjectResult>; +pub type ObjectDataStream = BoxStream<'static, ObjectResult>; diff --git a/src/object_store/src/object/opendal_engine/opendal_object_store.rs b/src/object_store/src/object/opendal_engine/opendal_object_store.rs index 204d9cac25753..408c42d38fc71 100644 --- a/src/object_store/src/object/opendal_engine/opendal_object_store.rs +++ b/src/object_store/src/object/opendal_engine/opendal_object_store.rs @@ -12,17 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::ops::Range; + use bytes::Bytes; use fail::fail_point; -use futures::{stream, StreamExt}; +use futures::{stream, StreamExt, TryStreamExt}; use opendal::services::Memory; use opendal::{Metakey, Operator, Writer}; use risingwave_common::range::RangeBoundsExt; -use tokio::io::AsyncRead; use crate::object::{ - BoxedStreamingUploader, ObjectError, ObjectMetadata, ObjectMetadataIter, ObjectRangeBounds, - ObjectResult, ObjectStore, StreamingUploader, + BoxedStreamingUploader, ObjectDataStream, ObjectError, ObjectMetadata, ObjectMetadataIter, + ObjectRangeBounds, ObjectResult, ObjectStore, StreamingUploader, }; /// Opendal object storage. @@ -107,22 +108,18 @@ impl ObjectStore for OpendalObjectStore { async fn streaming_read( &self, path: &str, - start_pos: Option, - ) -> ObjectResult> { + range: Range, + ) -> ObjectResult { fail_point!("opendal_streaming_read_err", |_| Err( ObjectError::internal("opendal streaming read error") )); - let reader = match start_pos { - Some(start_position) => { - self.op - .reader_with(path) - .range(start_position as u64..) - .await? - } - None => self.op.reader(path).await?, - }; + let range: Range = (range.start as u64)..(range.end as u64); + let reader = self.op.reader_with(path).range(range).await?; + let stream = reader + .into_stream() + .map(|item| item.map_err(|e| ObjectError::internal(format!("OpenDalError: {:?}", e)))); - Ok(Box::new(reader)) + Ok(Box::pin(stream)) } async fn metadata(&self, path: &str) -> ObjectResult { diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index bc826cbbd2ec8..899d1dc128ee2 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -14,6 +14,7 @@ use std::cmp; use std::collections::VecDeque; +use std::ops::Range; use std::pin::Pin; use std::sync::Arc; use std::task::{ready, Context, Poll}; @@ -30,6 +31,7 @@ use aws_sdk_s3::types::{ CompletedPart, Delete, ExpirationStatus, LifecycleRule, LifecycleRuleFilter, ObjectIdentifier, }; use aws_sdk_s3::Client; +use aws_smithy_http::futures_stream_adapter::FuturesStreamCompatByteStream; use aws_smithy_runtime::client::http::hyper_014::HyperClientBuilder; use aws_smithy_runtime_api::client::http::HttpClient; use aws_smithy_runtime_api::client::result::SdkError; @@ -38,13 +40,12 @@ use aws_smithy_types::retry::RetryConfig; use either::Either; use fail::fail_point; use futures::future::{try_join_all, BoxFuture, FutureExt}; -use futures::{stream, Stream}; +use futures::{stream, Stream, StreamExt, TryStreamExt}; use hyper::{Body, Response}; use itertools::Itertools; use risingwave_common::config::default::s3_objstore_config; use risingwave_common::monitor::connection::monitor_connector; use risingwave_common::range::RangeBoundsExt; -use tokio::io::AsyncRead; use tokio::task::JoinHandle; use tokio_retry::strategy::{jitter, ExponentialBackoff}; @@ -53,7 +54,7 @@ use super::{ BoxedStreamingUploader, Bytes, ObjectError, ObjectMetadata, ObjectRangeBounds, ObjectResult, ObjectStore, StreamingUploader, }; -use crate::object::{try_update_failure_metric, ObjectMetadataIter}; +use crate::object::{try_update_failure_metric, ObjectDataStream, ObjectMetadataIter}; type PartId = i32; @@ -429,8 +430,8 @@ impl ObjectStore for S3ObjectStore { async fn streaming_read( &self, path: &str, - start_pos: Option, - ) -> ObjectResult> { + range: Range, + ) -> ObjectResult { fail_point!("s3_streaming_read_err", |_| Err(ObjectError::internal( "s3 streaming read error" ))); @@ -439,11 +440,7 @@ impl ObjectStore for S3ObjectStore { let resp = tokio_retry::RetryIf::spawn( self.config.get_retry_strategy(), || async { - match self - .obj_store_request(path, start_pos.unwrap_or_default()..) - .send() - .await - { + match self.obj_store_request(path, range.clone()).send().await { Ok(resp) => Ok(resp), Err(err) => { if let SdkError::DispatchFailure(e) = &err @@ -462,8 +459,13 @@ impl ObjectStore for S3ObjectStore { Self::should_retry, ) .await?; + let reader = FuturesStreamCompatByteStream::new(resp.body); - Ok(Box::new(resp.body.into_async_read())) + Ok(Box::pin( + reader + .into_stream() + .map(|item| item.map_err(ObjectError::from)), + )) } /// Permanently deletes the whole object. @@ -528,6 +530,10 @@ impl ObjectStore for S3ObjectStore { fn store_media_type(&self) -> &'static str { "s3" } + + fn recv_buffer_size(&self) -> usize { + self.config.recv_buffer_size.unwrap_or(1 << 21) + } } impl S3ObjectStore { diff --git a/src/storage/benches/bench_compactor.rs b/src/storage/benches/bench_compactor.rs index 9e48c2093328a..16d4f603cc93a 100644 --- a/src/storage/benches/bench_compactor.rs +++ b/src/storage/benches/bench_compactor.rs @@ -56,6 +56,7 @@ pub fn mock_sstable_store() -> SstableStoreRef { 64 << 20, 128 << 20, 0, + 64 << 20, FileCache::none(), FileCache::none(), None, @@ -229,6 +230,7 @@ fn bench_merge_iterator_compactor(c: &mut Criterion) { let read_options = Arc::new(SstableIteratorReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), must_iterated_end_user_key: None, + max_preload_retry_times: 0, }); c.bench_function("bench_union_merge_iterator", |b| { b.to_async(FuturesExecutor).iter(|| { diff --git a/src/storage/benches/bench_multi_builder.rs b/src/storage/benches/bench_multi_builder.rs index 9bf0e0a9546ec..e84072d88207d 100644 --- a/src/storage/benches/bench_multi_builder.rs +++ b/src/storage/benches/bench_multi_builder.rs @@ -142,6 +142,7 @@ fn bench_builder( 64 << 20, 128 << 20, 0, + 64 << 20, FileCache::none(), FileCache::none(), None, diff --git a/src/storage/hummock_test/benches/bench_hummock_iter.rs b/src/storage/hummock_test/benches/bench_hummock_iter.rs index abac4d9b57c06..1a04a83730d88 100644 --- a/src/storage/hummock_test/benches/bench_hummock_iter.rs +++ b/src/storage/hummock_test/benches/bench_hummock_iter.rs @@ -109,7 +109,7 @@ fn criterion_benchmark(c: &mut Criterion) { epoch, ReadOptions { ignore_range_tombstone: true, - prefetch_options: PrefetchOptions::new_for_exhaust_iter(), + prefetch_options: PrefetchOptions::default(), cache_policy: CachePolicy::Fill(CachePriority::High), ..Default::default() }, diff --git a/src/storage/hummock_test/src/bin/replay/main.rs b/src/storage/hummock_test/src/bin/replay/main.rs index ae6038d8b5d16..d64f7dfbaf478 100644 --- a/src/storage/hummock_test/src/bin/replay/main.rs +++ b/src/storage/hummock_test/src/bin/replay/main.rs @@ -109,6 +109,7 @@ async fn create_replay_hummock(r: Record, args: &Args) -> Result>, end_bound: Bound>, truth: &ChaosTestTruth, @@ -940,14 +840,7 @@ mod tests { Unbounded => key_from_num(999999999999).into_bytes(), _ => unimplemented!(), }; - let cache = create_small_table_cache(); - let handle = cache.insert( - sstable.id, - sstable.id, - 1, - Box::new(sstable), - CachePriority::High, - ); + let backward_iters = vec![BackwardSstableIterator::new(handle, sstable_store)]; let bmi = UnorderedMergeIteratorInner::new(backward_iters); let mut bui = BackwardUserIterator::for_test(bmi, (start_bound, end_bound)); @@ -989,7 +882,7 @@ mod tests { type ChaosTestTruth = BTreeMap, BTreeMap, HummockValue>>; - async fn generate_chaos_test_data() -> (usize, Sstable, ChaosTestTruth, SstableStoreRef) { + async fn generate_chaos_test_data() -> (usize, TableHolder, ChaosTestTruth, SstableStoreRef) { // We first generate the key value pairs. let mut rng = thread_rng(); #[allow(clippy::mutable_key_type)] @@ -1187,16 +1080,7 @@ mod tests { ) .await; - let cache = create_small_table_cache(); - let handle0 = cache.insert( - table0.id, - table0.id, - 1, - Box::new(table0), - CachePriority::High, - ); - - let backward_iters = vec![BackwardSstableIterator::new(handle0, sstable_store)]; + let backward_iters = vec![BackwardSstableIterator::new(table0, sstable_store)]; let min_epoch = (TEST_KEYS_COUNT / 5) as u64; let mi = UnorderedMergeIteratorInner::new(backward_iters); diff --git a/src/storage/src/hummock/iterator/forward_concat.rs b/src/storage/src/hummock/iterator/forward_concat.rs index 736d4c0b42828..7a9ab721de9c6 100644 --- a/src/storage/src/hummock/iterator/forward_concat.rs +++ b/src/storage/src/hummock/iterator/forward_concat.rs @@ -24,7 +24,7 @@ mod tests { use super::*; use crate::hummock::iterator::test_utils::{ - default_builder_opt_for_test, gen_iterator_test_sstable_base, + default_builder_opt_for_test, gen_iterator_test_sstable_info, gen_iterator_test_sstable_with_range_tombstones, iterator_test_key_of, iterator_test_value_of, mock_sstable_store, TEST_KEYS_COUNT, }; @@ -34,7 +34,7 @@ mod tests { #[tokio::test] async fn test_concat_iterator() { let sstable_store = mock_sstable_store(); - let table0 = gen_iterator_test_sstable_base( + let table0 = gen_iterator_test_sstable_info( 0, default_builder_opt_for_test(), |x| x, @@ -42,7 +42,7 @@ mod tests { TEST_KEYS_COUNT, ) .await; - let table1 = gen_iterator_test_sstable_base( + let table1 = gen_iterator_test_sstable_info( 1, default_builder_opt_for_test(), |x| TEST_KEYS_COUNT + x, @@ -50,7 +50,7 @@ mod tests { TEST_KEYS_COUNT, ) .await; - let table2 = gen_iterator_test_sstable_base( + let table2 = gen_iterator_test_sstable_info( 2, default_builder_opt_for_test(), |x| TEST_KEYS_COUNT * 2 + x, @@ -59,11 +59,7 @@ mod tests { ) .await; let mut iter = ConcatIterator::new( - vec![ - table0.get_sstable_info(), - table1.get_sstable_info(), - table2.get_sstable_info(), - ], + vec![table0, table1, table2], sstable_store, Arc::new(SstableIteratorReadOptions::default()), ); @@ -99,7 +95,7 @@ mod tests { #[tokio::test] async fn test_concat_seek() { let sstable_store = mock_sstable_store(); - let table0 = gen_iterator_test_sstable_base( + let table0 = gen_iterator_test_sstable_info( 0, default_builder_opt_for_test(), |x| x, @@ -107,7 +103,7 @@ mod tests { TEST_KEYS_COUNT, ) .await; - let table1 = gen_iterator_test_sstable_base( + let table1 = gen_iterator_test_sstable_info( 1, default_builder_opt_for_test(), |x| TEST_KEYS_COUNT + x, @@ -115,7 +111,7 @@ mod tests { TEST_KEYS_COUNT, ) .await; - let table2 = gen_iterator_test_sstable_base( + let table2 = gen_iterator_test_sstable_info( 2, default_builder_opt_for_test(), |x| TEST_KEYS_COUNT * 2 + x, @@ -124,11 +120,7 @@ mod tests { ) .await; let mut iter = ConcatIterator::new( - vec![ - table0.get_sstable_info(), - table1.get_sstable_info(), - table2.get_sstable_info(), - ], + vec![table0, table1, table2], sstable_store, Arc::new(SstableIteratorReadOptions::default()), ); @@ -178,7 +170,7 @@ mod tests { #[tokio::test] async fn test_concat_seek_not_exists() { let sstable_store = mock_sstable_store(); - let table0 = gen_iterator_test_sstable_base( + let table0 = gen_iterator_test_sstable_info( 0, default_builder_opt_for_test(), |x| x * 2, @@ -186,7 +178,7 @@ mod tests { TEST_KEYS_COUNT, ) .await; - let table1 = gen_iterator_test_sstable_base( + let table1 = gen_iterator_test_sstable_info( 1, default_builder_opt_for_test(), |x| (TEST_KEYS_COUNT + x) * 2, @@ -194,7 +186,7 @@ mod tests { TEST_KEYS_COUNT, ) .await; - let table2 = gen_iterator_test_sstable_base( + let table2 = gen_iterator_test_sstable_info( 2, default_builder_opt_for_test(), |x| (2 * TEST_KEYS_COUNT + x) * 2, @@ -203,11 +195,7 @@ mod tests { ) .await; let mut iter = ConcatIterator::new( - vec![ - table0.get_sstable_info(), - table1.get_sstable_info(), - table2.get_sstable_info(), - ], + vec![table0, table1, table2], sstable_store, Arc::new(SstableIteratorReadOptions::default()), ); @@ -255,7 +243,7 @@ mod tests { sstable_store.clone(), ) .await; - let table3 = gen_iterator_test_sstable_base( + let table3 = gen_iterator_test_sstable_info( 3, default_builder_opt_for_test(), |x| TEST_KEYS_COUNT + x, @@ -264,11 +252,7 @@ mod tests { ) .await; let mut iter = ConcatIterator::new( - vec![ - table1.get_sstable_info(), - table2.get_sstable_info(), - table3.get_sstable_info(), - ], + vec![table1, table2, table3], sstable_store, Arc::new(SstableIteratorReadOptions::default()), ); diff --git a/src/storage/src/hummock/iterator/forward_merge.rs b/src/storage/src/hummock/iterator/forward_merge.rs index deffb1472c483..56aa73a078f58 100644 --- a/src/storage/src/hummock/iterator/forward_merge.rs +++ b/src/storage/src/hummock/iterator/forward_merge.rs @@ -20,12 +20,11 @@ mod test { use std::task::Poll; use futures::{pin_mut, FutureExt}; - use risingwave_common::cache::CachePriority; use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKey}; use risingwave_hummock_sdk::EpochWithGap; use crate::hummock::iterator::test_utils::{ - default_builder_opt_for_test, gen_iterator_test_sstable_base, + default_builder_opt_for_test, gen_iterator_test_sstable_info, gen_merge_iterator_interleave_test_sstable_iters, iterator_test_key_of, iterator_test_value_of, mock_sstable_store, TEST_KEYS_COUNT, }; @@ -36,7 +35,7 @@ mod test { use crate::hummock::sstable::{ SstableIterator, SstableIteratorReadOptions, SstableIteratorType, }; - use crate::hummock::test_utils::{create_small_table_cache, gen_test_sstable}; + use crate::hummock::test_utils::gen_test_sstable; use crate::hummock::value::HummockValue; use crate::hummock::HummockResult; use crate::monitor::StoreLocalStatistic; @@ -146,41 +145,36 @@ mod test { async fn test_merge_invalidate_reset() { let sstable_store = mock_sstable_store(); let read_options = Arc::new(SstableIteratorReadOptions::default()); - let table0 = Box::new( - gen_iterator_test_sstable_base( - 0, - default_builder_opt_for_test(), - |x| x, - sstable_store.clone(), - TEST_KEYS_COUNT, - ) - .await, - ); - let table1 = Box::new( - gen_iterator_test_sstable_base( - 1, - default_builder_opt_for_test(), - |x| TEST_KEYS_COUNT + x, - sstable_store.clone(), - TEST_KEYS_COUNT, - ) - .await, - ); - - let cache = create_small_table_cache(); - + let table0 = gen_iterator_test_sstable_info( + 0, + default_builder_opt_for_test(), + |x| x, + sstable_store.clone(), + TEST_KEYS_COUNT, + ) + .await; + let table1 = gen_iterator_test_sstable_info( + 1, + default_builder_opt_for_test(), + |x| TEST_KEYS_COUNT + x, + sstable_store.clone(), + TEST_KEYS_COUNT, + ) + .await; + + let mut stats = StoreLocalStatistic::default(); let mut unordered_iter: HummockIteratorUnion< Forward, UnorderedMergeIteratorInner, OrderedMergeIteratorInner, > = HummockIteratorUnion::First(UnorderedMergeIteratorInner::new(vec![ SstableIterator::create( - cache.insert(table0.id, table0.id, 1, table0, CachePriority::High), + sstable_store.sstable(&table0, &mut stats).await.unwrap(), sstable_store.clone(), read_options.clone(), ), SstableIterator::create( - cache.insert(table1.id, table1.id, 1, table1, CachePriority::High), + sstable_store.sstable(&table1, &mut stats).await.unwrap(), sstable_store.clone(), read_options.clone(), ), @@ -191,12 +185,12 @@ mod test { OrderedMergeIteratorInner, > = HummockIteratorUnion::Second(OrderedMergeIteratorInner::new(vec![ SstableIterator::create( - cache.lookup(0, &0).unwrap(), + sstable_store.sstable(&table0, &mut stats).await.unwrap(), sstable_store.clone(), read_options.clone(), ), SstableIterator::create( - cache.lookup(1, &1).unwrap(), + sstable_store.sstable(&table1, &mut stats).await.unwrap(), sstable_store.clone(), read_options.clone(), ), @@ -229,83 +223,57 @@ mod test { let sstable_store = mock_sstable_store(); let read_options = Arc::new(SstableIteratorReadOptions::default()); - let non_overlapped_sstable = Box::new( - gen_test_sstable( - default_builder_opt_for_test(), - 0, - (0..TEST_KEYS_COUNT).filter(|x| x % 3 == 0).map(|x| { - ( - iterator_test_key_of(x), - HummockValue::put(format!("non_overlapped_{}", x).as_bytes().to_vec()), - ) - }), - sstable_store.clone(), - ) - .await, - ); - - let overlapped_old_sstable = Box::new( - gen_test_sstable( - default_builder_opt_for_test(), - 1, - (0..TEST_KEYS_COUNT).filter(|x| x % 3 != 0).map(|x| { - ( - iterator_test_key_of(x), - HummockValue::put(format!("overlapped_old_{}", x).as_bytes().to_vec()), - ) - }), - sstable_store.clone(), - ) - .await, - ); - - let overlapped_new_sstable = Box::new( - gen_test_sstable( - default_builder_opt_for_test(), - 2, - (0..TEST_KEYS_COUNT).filter(|x| x % 3 == 1).map(|x| { - ( - iterator_test_key_of(x), - HummockValue::put(format!("overlapped_new_{}", x).as_bytes().to_vec()), - ) - }), - sstable_store.clone(), - ) - .await, - ); - let cache = create_small_table_cache(); - + let non_overlapped_sstable = gen_test_sstable( + default_builder_opt_for_test(), + 0, + (0..TEST_KEYS_COUNT).filter(|x| x % 3 == 0).map(|x| { + ( + iterator_test_key_of(x), + HummockValue::put(format!("non_overlapped_{}", x).as_bytes().to_vec()), + ) + }), + sstable_store.clone(), + ) + .await; + + let overlapped_old_sstable = gen_test_sstable( + default_builder_opt_for_test(), + 1, + (0..TEST_KEYS_COUNT).filter(|x| x % 3 != 0).map(|x| { + ( + iterator_test_key_of(x), + HummockValue::put(format!("overlapped_old_{}", x).as_bytes().to_vec()), + ) + }), + sstable_store.clone(), + ) + .await; + + let overlapped_new_sstable = gen_test_sstable( + default_builder_opt_for_test(), + 2, + (0..TEST_KEYS_COUNT).filter(|x| x % 3 == 1).map(|x| { + ( + iterator_test_key_of(x), + HummockValue::put(format!("overlapped_new_{}", x).as_bytes().to_vec()), + ) + }), + sstable_store.clone(), + ) + .await; let mut iter = OrderedMergeIteratorInner::new(vec![ SstableIterator::create( - cache.insert( - non_overlapped_sstable.id, - non_overlapped_sstable.id, - 1, - non_overlapped_sstable, - CachePriority::High, - ), + non_overlapped_sstable, sstable_store.clone(), read_options.clone(), ), SstableIterator::create( - cache.insert( - overlapped_new_sstable.id, - overlapped_new_sstable.id, - 1, - overlapped_new_sstable, - CachePriority::High, - ), + overlapped_new_sstable, sstable_store.clone(), read_options.clone(), ), SstableIterator::create( - cache.insert( - overlapped_old_sstable.id, - overlapped_old_sstable.id, - 1, - overlapped_old_sstable, - CachePriority::High, - ), + overlapped_old_sstable, sstable_store.clone(), read_options.clone(), ), diff --git a/src/storage/src/hummock/iterator/forward_user.rs b/src/storage/src/hummock/iterator/forward_user.rs index 47e58e2516253..e26675dc11979 100644 --- a/src/storage/src/hummock/iterator/forward_user.rs +++ b/src/storage/src/hummock/iterator/forward_user.rs @@ -306,8 +306,6 @@ mod tests { use std::ops::Bound::*; use std::sync::Arc; - use risingwave_common::cache::CachePriority; - use super::*; use crate::hummock::iterator::test_utils::{ default_builder_opt_for_test, gen_iterator_test_sstable_base, @@ -321,9 +319,8 @@ mod tests { SstableIterator, SstableIteratorReadOptions, SstableIteratorType, }; use crate::hummock::sstable_store::SstableStoreRef; - use crate::hummock::test_utils::create_small_table_cache; use crate::hummock::value::HummockValue; - use crate::hummock::{Sstable, SstableDeleteRangeIterator}; + use crate::hummock::{SstableDeleteRangeIterator, TableHolder}; #[tokio::test] async fn test_basic() { @@ -353,41 +350,10 @@ mod tests { TEST_KEYS_COUNT, ) .await; - let cache = create_small_table_cache(); let iters = vec![ - SstableIterator::create( - cache.insert( - table0.id, - table0.id, - 1, - Box::new(table0), - CachePriority::High, - ), - sstable_store.clone(), - read_options.clone(), - ), - SstableIterator::create( - cache.insert( - table1.id, - table1.id, - 1, - Box::new(table1), - CachePriority::High, - ), - sstable_store.clone(), - read_options.clone(), - ), - SstableIterator::create( - cache.insert( - table2.id, - table2.id, - 1, - Box::new(table2), - CachePriority::High, - ), - sstable_store, - read_options.clone(), - ), + SstableIterator::create(table0, sstable_store.clone(), read_options.clone()), + SstableIterator::create(table1, sstable_store.clone(), read_options.clone()), + SstableIterator::create(table2, sstable_store, read_options.clone()), ]; let mi = UnorderedMergeIteratorInner::new(iters); @@ -438,41 +404,10 @@ mod tests { ) .await; let read_options = Arc::new(SstableIteratorReadOptions::default()); - let cache = create_small_table_cache(); let iters = vec![ - SstableIterator::create( - cache.insert( - table0.id, - table0.id, - 1, - Box::new(table0), - CachePriority::High, - ), - sstable_store.clone(), - read_options.clone(), - ), - SstableIterator::create( - cache.insert( - table1.id, - table1.id, - 1, - Box::new(table1), - CachePriority::High, - ), - sstable_store.clone(), - read_options.clone(), - ), - SstableIterator::create( - cache.insert( - table2.id, - table2.id, - 1, - Box::new(table2), - CachePriority::High, - ), - sstable_store, - read_options, - ), + SstableIterator::create(table0, sstable_store.clone(), read_options.clone()), + SstableIterator::create(table1, sstable_store.clone(), read_options.clone()), + SstableIterator::create(table2, sstable_store, read_options), ]; let mi = UnorderedMergeIteratorInner::new(iters); @@ -533,30 +468,9 @@ mod tests { gen_iterator_test_sstable_from_kv_pair(1, kv_pairs, sstable_store.clone()).await; let read_options = Arc::new(SstableIteratorReadOptions::default()); - let cache = create_small_table_cache(); let iters = vec![ - SstableIterator::create( - cache.insert( - table0.id, - table0.id, - 1, - Box::new(table0), - CachePriority::High, - ), - sstable_store.clone(), - read_options.clone(), - ), - SstableIterator::create( - cache.insert( - table1.id, - table1.id, - 1, - Box::new(table1), - CachePriority::High, - ), - sstable_store.clone(), - read_options, - ), + SstableIterator::create(table0, sstable_store.clone(), read_options.clone()), + SstableIterator::create(table1, sstable_store.clone(), read_options), ]; let mi = UnorderedMergeIteratorInner::new(iters); @@ -577,7 +491,7 @@ mod tests { async fn generate_test_data( sstable_store: SstableStoreRef, range_tombstones: Vec<(usize, usize, u64)>, - ) -> Sstable { + ) -> TableHolder { let kv_pairs = vec![ (0, 200, HummockValue::delete()), (0, 100, HummockValue::put(iterator_test_value_of(0))), @@ -594,13 +508,17 @@ mod tests { (7, 100, HummockValue::put(iterator_test_value_of(7))), (8, 100, HummockValue::put(iterator_test_value_of(8))), ]; - gen_iterator_test_sstable_with_range_tombstones( + let sst_info = gen_iterator_test_sstable_with_range_tombstones( 0, kv_pairs, range_tombstones, - sstable_store, + sstable_store.clone(), ) - .await + .await; + sstable_store + .sstable(&sst_info, &mut StoreLocalStatistic::default()) + .await + .unwrap() } // left..=end @@ -609,13 +527,8 @@ mod tests { let sstable_store = mock_sstable_store(); // key=[idx, epoch], value let table = generate_test_data(sstable_store.clone(), vec![]).await; - let cache = create_small_table_cache(); let read_options = Arc::new(SstableIteratorReadOptions::default()); - let iters = vec![SstableIterator::create( - cache.insert(table.id, table.id, 1, Box::new(table), CachePriority::High), - sstable_store, - read_options, - )]; + let iters = vec![SstableIterator::create(table, sstable_store, read_options)]; let mi = UnorderedMergeIteratorInner::new(iters); let begin_key = Included(iterator_test_bytes_user_key_of(2)); @@ -692,13 +605,8 @@ mod tests { ]; let table = gen_iterator_test_sstable_from_kv_pair(0, kv_pairs, sstable_store.clone()).await; - let cache = create_small_table_cache(); let read_options = Arc::new(SstableIteratorReadOptions::default()); - let iters = vec![SstableIterator::create( - cache.insert(table.id, table.id, 1, Box::new(table), CachePriority::High), - sstable_store, - read_options, - )]; + let iters = vec![SstableIterator::create(table, sstable_store, read_options)]; let mi = UnorderedMergeIteratorInner::new(iters); let begin_key = Included(iterator_test_bytes_user_key_of(2)); @@ -760,13 +668,8 @@ mod tests { // key=[idx, epoch], value let table = generate_test_data(sstable_store.clone(), vec![]).await; - let cache = create_small_table_cache(); let read_options = Arc::new(SstableIteratorReadOptions::default()); - let iters = vec![SstableIterator::create( - cache.insert(table.id, table.id, 1, Box::new(table), CachePriority::High), - sstable_store, - read_options, - )]; + let iters = vec![SstableIterator::create(table, sstable_store, read_options)]; let mi = UnorderedMergeIteratorInner::new(iters); let end_key = Included(iterator_test_bytes_user_key_of(7)); @@ -829,13 +732,8 @@ mod tests { let sstable_store = mock_sstable_store(); // key=[idx, epoch], value let table = generate_test_data(sstable_store.clone(), vec![]).await; - let cache = create_small_table_cache(); let read_options = Arc::new(SstableIteratorReadOptions::default()); - let iters = vec![SstableIterator::create( - cache.insert(table.id, table.id, 1, Box::new(table), CachePriority::High), - sstable_store, - read_options, - )]; + let iters = vec![SstableIterator::create(table, sstable_store, read_options)]; let mi = UnorderedMergeIteratorInner::new(iters); let begin_key = Included(iterator_test_bytes_user_key_of(2)); @@ -909,15 +807,8 @@ mod tests { 1, ) .await; - let cache = create_small_table_cache(); let iters = vec![SstableIterator::create( - cache.insert( - table0.id, - table0.id, - 1, - Box::new(table0), - CachePriority::High, - ), + table0, sstable_store.clone(), read_options.clone(), )]; @@ -951,20 +842,16 @@ mod tests { vec![(0, 2, 300), (1, 4, 150), (3, 6, 50), (5, 8, 150)], ) .await; - let cache = create_small_table_cache(); let read_options = SstableIteratorReadOptions::default(); - let table_id = table.id; let iters = vec![SstableIterator::create( - cache.insert(table.id, table.id, 1, Box::new(table), CachePriority::High), + table.clone(), sstable_store.clone(), Arc::new(read_options), )]; let mi = UnorderedMergeIteratorInner::new(iters); let mut del_iter = ForwardMergeRangeIterator::new(150); - del_iter.add_sst_iter(SstableDeleteRangeIterator::new( - cache.lookup(table_id, &table_id).unwrap(), - )); + del_iter.add_sst_iter(SstableDeleteRangeIterator::new(table.clone())); let mut ui: UserIterator<_> = UserIterator::new(mi, (Unbounded, Unbounded), 150, 0, None, del_iter); @@ -987,14 +874,12 @@ mod tests { let read_options = SstableIteratorReadOptions::default(); let iters = vec![SstableIterator::create( - cache.lookup(table_id, &table_id).unwrap(), + table.clone(), sstable_store, Arc::new(read_options), )]; let mut del_iter = ForwardMergeRangeIterator::new(300); - del_iter.add_sst_iter(SstableDeleteRangeIterator::new( - cache.lookup(table_id, &table_id).unwrap(), - )); + del_iter.add_sst_iter(SstableDeleteRangeIterator::new(table.clone())); let mi = UnorderedMergeIteratorInner::new(iters); let mut ui: UserIterator<_> = UserIterator::new(mi, (Unbounded, Unbounded), 300, 0, None, del_iter); diff --git a/src/storage/src/hummock/iterator/test_utils.rs b/src/storage/src/hummock/iterator/test_utils.rs index 18ce9791797de..b01233a1664b4 100644 --- a/src/storage/src/hummock/iterator/test_utils.rs +++ b/src/storage/src/hummock/iterator/test_utils.rs @@ -17,23 +17,23 @@ use std::sync::Arc; use bytes::Bytes; use itertools::Itertools; -use risingwave_common::cache::CachePriority; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKey}; use risingwave_hummock_sdk::{EpochWithGap, HummockEpoch, HummockSstableObjectId}; use risingwave_object_store::object::{ InMemObjectStore, ObjectStore, ObjectStoreImpl, ObjectStoreRef, }; +use risingwave_pb::hummock::SstableInfo; use crate::hummock::sstable::SstableIteratorReadOptions; use crate::hummock::sstable_store::SstableStore; pub use crate::hummock::test_utils::default_builder_opt_for_test; use crate::hummock::test_utils::{ - create_small_table_cache, gen_test_sstable, gen_test_sstable_with_range_tombstone, + gen_test_sstable, gen_test_sstable_info, gen_test_sstable_with_range_tombstone, }; use crate::hummock::{ - DeleteRangeTombstone, FileCache, HummockValue, Sstable, SstableBuilderOptions, SstableIterator, - SstableIteratorType, SstableStoreRef, + DeleteRangeTombstone, FileCache, HummockValue, SstableBuilderOptions, SstableIterator, + SstableIteratorType, SstableStoreRef, TableHolder, }; use crate::monitor::ObjectStoreMetrics; @@ -65,6 +65,7 @@ pub fn mock_sstable_store_with_object_store(store: ObjectStoreRef) -> SstableSto 64 << 20, 64 << 20, 0, + 64 << 20, FileCache::none(), FileCache::none(), None, @@ -126,6 +127,30 @@ pub fn transform_shared_buffer( .collect_vec() } +/// Generates a test table used in almost all table-related tests. Developers may verify the +/// correctness of their implementations by comparing the got value and the expected value +/// generated by `test_key_of` and `test_value_of`. +pub async fn gen_iterator_test_sstable_info( + object_id: HummockSstableObjectId, + opts: SstableBuilderOptions, + idx_mapping: impl Fn(usize) -> usize, + sstable_store: SstableStoreRef, + total: usize, +) -> SstableInfo { + gen_test_sstable_info( + opts, + object_id, + (0..total).map(|i| { + ( + iterator_test_key_of(idx_mapping(i)), + HummockValue::put(iterator_test_value_of(idx_mapping(i))), + ) + }), + sstable_store, + ) + .await +} + /// Generates a test table used in almost all table-related tests. Developers may verify the /// correctness of their implementations by comparing the got value and the expected value /// generated by `test_key_of` and `test_value_of`. @@ -135,7 +160,7 @@ pub async fn gen_iterator_test_sstable_base( idx_mapping: impl Fn(usize) -> usize, sstable_store: SstableStoreRef, total: usize, -) -> Sstable { +) -> TableHolder { gen_test_sstable( opts, object_id, @@ -155,7 +180,7 @@ pub async fn gen_iterator_test_sstable_from_kv_pair( object_id: HummockSstableObjectId, kv_pairs: Vec<(usize, u64, HummockValue>)>, sstable_store: SstableStoreRef, -) -> Sstable { +) -> TableHolder { gen_test_sstable( default_builder_opt_for_test(), object_id, @@ -173,7 +198,7 @@ pub async fn gen_iterator_test_sstable_with_range_tombstones( kv_pairs: Vec<(usize, u64, HummockValue>)>, delete_ranges: Vec<(usize, usize, u64)>, sstable_store: SstableStoreRef, -) -> Sstable { +) -> SstableInfo { let range_tombstones = delete_ranges .into_iter() .map(|(start, end, epoch)| { @@ -204,7 +229,6 @@ pub async fn gen_merge_iterator_interleave_test_sstable_iters( count: usize, ) -> Vec { let sstable_store = mock_sstable_store(); - let cache = create_small_table_cache(); let mut result = vec![]; for i in 0..count { let table = gen_iterator_test_sstable_base( @@ -215,9 +239,8 @@ pub async fn gen_merge_iterator_interleave_test_sstable_iters( key_count, ) .await; - let handle = cache.insert(table.id, table.id, 1, Box::new(table), CachePriority::High); result.push(SstableIterator::create( - handle, + table, sstable_store.clone(), Arc::new(SstableIteratorReadOptions::default()), )); @@ -232,7 +255,7 @@ pub async fn gen_iterator_test_sstable_with_incr_epoch( sstable_store: SstableStoreRef, total: usize, epoch_base: u64, -) -> Sstable { +) -> TableHolder { gen_test_sstable( opts, object_id, diff --git a/src/storage/src/hummock/sstable/backward_sstable_iterator.rs b/src/storage/src/hummock/sstable/backward_sstable_iterator.rs index ac40950a419d7..36c4e166eae90 100644 --- a/src/storage/src/hummock/sstable/backward_sstable_iterator.rs +++ b/src/storage/src/hummock/sstable/backward_sstable_iterator.rs @@ -184,22 +184,20 @@ mod tests { use crate::assert_bytes_eq; use crate::hummock::iterator::test_utils::mock_sstable_store; use crate::hummock::test_utils::{ - create_small_table_cache, default_builder_opt_for_test, gen_default_test_sstable, - test_key_of, test_value_of, TEST_KEYS_COUNT, + default_builder_opt_for_test, gen_default_test_sstable, test_key_of, test_value_of, + TEST_KEYS_COUNT, }; #[tokio::test] async fn test_backward_sstable_iterator() { // build remote sstable let sstable_store = mock_sstable_store(); - let sstable = + let handle = gen_default_test_sstable(default_builder_opt_for_test(), 0, sstable_store.clone()) .await; // We should have at least 10 blocks, so that sstable iterator test could cover more code // path. - assert!(sstable.meta.block_metas.len() > 10); - let cache = create_small_table_cache(); - let handle = cache.insert(0, 0, 1, Box::new(sstable), CachePriority::High); + assert!(handle.value().meta.block_metas.len() > 10); let mut sstable_iter = BackwardSstableIterator::new(handle, sstable_store); let mut cnt = TEST_KEYS_COUNT; sstable_iter.rewind().await.unwrap(); @@ -224,10 +222,8 @@ mod tests { .await; // We should have at least 10 blocks, so that sstable iterator test could cover more code // path. - assert!(sstable.meta.block_metas.len() > 10); - let cache = create_small_table_cache(); - let handle = cache.insert(0, 0, 1, Box::new(sstable), CachePriority::High); - let mut sstable_iter = BackwardSstableIterator::new(handle, sstable_store); + assert!(sstable.value().meta.block_metas.len() > 10); + let mut sstable_iter = BackwardSstableIterator::new(sstable, sstable_store); let mut all_key_to_test = (0..TEST_KEYS_COUNT).collect_vec(); let mut rng = thread_rng(); all_key_to_test.shuffle(&mut rng); diff --git a/src/storage/src/hummock/sstable/delete_range_aggregator.rs b/src/storage/src/hummock/sstable/delete_range_aggregator.rs index 2eac06a70a724..00ab5a1e47b1a 100644 --- a/src/storage/src/hummock/sstable/delete_range_aggregator.rs +++ b/src/storage/src/hummock/sstable/delete_range_aggregator.rs @@ -315,6 +315,7 @@ mod tests { mock_sstable_store, }; use crate::hummock::test_utils::{test_user_key, CompactionDeleteRangesBuilder}; + use crate::monitor::StoreLocalStatistic; #[tokio::test] pub async fn test_compaction_delete_range_iterator() { @@ -524,35 +525,39 @@ mod tests { async fn test_delete_range_get() { let sstable_store = mock_sstable_store(); // key=[idx, epoch], value - let sstable = gen_iterator_test_sstable_with_range_tombstones( + let sst_info = gen_iterator_test_sstable_with_range_tombstones( 0, vec![], vec![(0, 2, 300), (1, 4, 150), (3, 6, 50), (5, 8, 150)], - sstable_store, + sstable_store.clone(), ) .await; + let sstable = sstable_store + .sstable(&sst_info, &mut StoreLocalStatistic::default()) + .await + .unwrap(); let ret = get_min_delete_range_epoch_from_sstable( - &sstable, + sstable.value(), iterator_test_user_key_of(0).as_ref(), ); assert_eq!(ret, 300); let ret = get_min_delete_range_epoch_from_sstable( - &sstable, + sstable.value(), iterator_test_user_key_of(1).as_ref(), ); assert_eq!(ret, 150); let ret = get_min_delete_range_epoch_from_sstable( - &sstable, + sstable.value(), iterator_test_user_key_of(3).as_ref(), ); assert_eq!(ret, 50); let ret = get_min_delete_range_epoch_from_sstable( - &sstable, + sstable.value(), iterator_test_user_key_of(6).as_ref(), ); assert_eq!(ret, 150); let ret = get_min_delete_range_epoch_from_sstable( - &sstable, + sstable.value(), iterator_test_user_key_of(8).as_ref(), ); assert_eq!(ret, MAX_EPOCH); diff --git a/src/storage/src/hummock/sstable/forward_sstable_iterator.rs b/src/storage/src/hummock/sstable/forward_sstable_iterator.rs index 3988d082177f8..f7639cddbda57 100644 --- a/src/storage/src/hummock/sstable/forward_sstable_iterator.rs +++ b/src/storage/src/hummock/sstable/forward_sstable_iterator.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::cmp::Ordering::{Equal, Less}; -use std::collections::VecDeque; use std::future::Future; use std::ops::Bound::*; use std::sync::Arc; @@ -21,13 +20,9 @@ use std::sync::Arc; use risingwave_hummock_sdk::key::FullKey; use super::super::{HummockResult, HummockValue}; -use super::Sstable; use crate::hummock::iterator::{Forward, HummockIterator}; use crate::hummock::sstable::SstableIteratorReadOptions; -use crate::hummock::{ - BlockHolder, BlockIterator, BlockResponse, CachePolicy, SstableStore, SstableStoreRef, - TableHolder, -}; +use crate::hummock::{BatchBlockStream, BlockIterator, SstableStoreRef, TableHolder}; use crate::monitor::StoreLocalStatistic; pub trait SstableIteratorType: HummockIterator + 'static { @@ -38,106 +33,6 @@ pub trait SstableIteratorType: HummockIterator + 'static { ) -> Self; } -/// Prefetching may increase the memory footprint of the CN process because the prefetched blocks -/// cannot be evicted. -enum BlockFetcher { - Simple(SimpleFetchContext), - Prefetch(PrefetchContext), -} - -impl BlockFetcher { - async fn get_block( - &mut self, - sst: &Sstable, - block_idx: usize, - sstable_store: &SstableStore, - stats: &mut StoreLocalStatistic, - ) -> HummockResult { - match self { - BlockFetcher::Simple(context) => { - sstable_store - .get(sst, block_idx, context.cache_policy, stats) - .await - } - BlockFetcher::Prefetch(context) => { - context - .get_block(sst, block_idx, sstable_store, stats) - .await - } - } - } -} - -struct SimpleFetchContext { - cache_policy: CachePolicy, -} - -struct PrefetchContext { - prefetched_blocks: VecDeque<(usize, BlockResponse)>, - - /// block[cur_idx..=dest_idx] will definitely be visited in the future. - dest_idx: usize, - - cache_policy: CachePolicy, -} - -const DEFAULT_PREFETCH_BLOCK_NUM: usize = 1; - -impl PrefetchContext { - fn new(dest_idx: usize, cache_policy: CachePolicy) -> Self { - Self { - prefetched_blocks: VecDeque::with_capacity(DEFAULT_PREFETCH_BLOCK_NUM + 1), - dest_idx, - cache_policy, - } - } - - async fn get_block( - &mut self, - sst: &Sstable, - idx: usize, - sstable_store: &SstableStore, - stats: &mut StoreLocalStatistic, - ) -> HummockResult { - let is_empty = if let Some((prefetched_idx, _)) = self.prefetched_blocks.front() { - if *prefetched_idx == idx { - false - } else { - tracing::warn!(target: "events::storage::sstable::block_seek", "prefetch mismatch: sstable_object_id = {}, block_id = {}, prefetched_block_id = {}", sst.id, idx, *prefetched_idx); - self.prefetched_blocks.clear(); - true - } - } else { - true - }; - if is_empty { - self.prefetched_blocks.push_back(( - idx, - sstable_store - .get_block_response(sst, idx, self.cache_policy, stats) - .await?, - )); - } - let block_response = self.prefetched_blocks.pop_front().unwrap().1; - - let next_prefetch_idx = self - .prefetched_blocks - .back() - .map_or(idx, |(latest_idx, _)| *latest_idx) - + 1; - if next_prefetch_idx <= self.dest_idx { - self.prefetched_blocks.push_back(( - next_prefetch_idx, - sstable_store - .get_block_response(sst, next_prefetch_idx, self.cache_policy, stats) - .await?, - )); - } - - block_response.wait().await - } -} - /// Iterates on a sstable. pub struct SstableIterator { /// The iterator of the current block. @@ -146,11 +41,11 @@ pub struct SstableIterator { /// Current block index. cur_idx: usize, - /// simple or prefetch strategy - block_fetcher: BlockFetcher, - + preload_stream: Option, /// Reference to the sst pub sst: TableHolder, + preload_end_block_idx: usize, + preload_retry_times: usize, sstable_store: SstableStoreRef, stats: StoreLocalStatistic, @@ -166,60 +61,38 @@ impl SstableIterator { Self { block_iter: None, cur_idx: 0, - block_fetcher: BlockFetcher::Simple(SimpleFetchContext { - cache_policy: options.cache_policy, - }), + preload_stream: None, sst: sstable, sstable_store, stats: StoreLocalStatistic::default(), options, + preload_end_block_idx: 0, + preload_retry_times: 0, } } - fn init_block_fetcher(&mut self, start_idx: usize) { + fn init_block_prefetch_range(&mut self, start_idx: usize) { if let Some(bound) = self.options.must_iterated_end_user_key.as_ref() { let block_metas = &self.sst.value().meta.block_metas; let next_to_start_idx = start_idx + 1; if next_to_start_idx < block_metas.len() { - let dest_idx = match bound { - Unbounded => block_metas.len() - 1, // will not overflow + let end_idx = match bound { + Unbounded => block_metas.len(), Included(dest_key) => { let dest_key = dest_key.as_ref(); - if FullKey::decode(&block_metas[next_to_start_idx].smallest_key).user_key - > dest_key - { - start_idx - } else { - next_to_start_idx - + block_metas[(next_to_start_idx + 1)..].partition_point( - |block_meta| { - FullKey::decode(&block_meta.smallest_key).user_key - <= dest_key - }, - ) - } + block_metas.partition_point(|block_meta| { + FullKey::decode(&block_meta.smallest_key).user_key <= dest_key + }) } Excluded(end_key) => { let end_key = end_key.as_ref(); - if FullKey::decode(&block_metas[next_to_start_idx].smallest_key).user_key - >= end_key - { - start_idx - } else { - next_to_start_idx - + block_metas[(next_to_start_idx + 1)..].partition_point( - |block_meta| { - FullKey::decode(&block_meta.smallest_key).user_key < end_key - }, - ) - } + block_metas.partition_point(|block_meta| { + FullKey::decode(&block_meta.smallest_key).user_key < end_key + }) } }; - if start_idx < dest_idx { - self.block_fetcher = BlockFetcher::Prefetch(PrefetchContext::new( - dest_idx, - self.options.cache_policy, - )); + if start_idx + 1 < end_idx { + self.preload_end_block_idx = end_idx; } } } @@ -247,24 +120,95 @@ impl SstableIterator { // do cooperative scheduling. tokio::task::consume_budget().await; + let mut hit_cache = false; if idx >= self.sst.value().block_count() { self.block_iter = None; - } else { + return Ok(()); + } + // Maybe the previous preload stream breaks on some cached block, so here we can try to preload some data again + if self.preload_stream.is_none() && idx + 1 < self.preload_end_block_idx + && let Ok(preload_stream) = self.sstable_store + .preload_blocks(self.sst.value(), idx, self.preload_end_block_idx) + .await + { + self.preload_stream = preload_stream; + } + if self + .preload_stream + .as_ref() + .map(|preload_stream| preload_stream.next_block_index() <= idx) + .unwrap_or(false) + { + while let Some(preload_stream) = self.preload_stream.as_mut() { + let mut ret = Ok(()); + while preload_stream.next_block_index() < idx { + if let Err(e) = preload_stream.next_block().await { + ret = Err(e); + break; + } + } + if ret.is_ok() { + match preload_stream.next_block().await { + Ok(Some(block)) => { + hit_cache = true; + self.block_iter = Some(BlockIterator::new(block)); + break; + } + Ok(None) => { + self.preload_stream.take(); + break; + } + Err(e) => { + self.preload_stream.take(); + ret = Err(e); + } + } + } else { + self.preload_stream.take(); + } + if let Err(e) = ret { + assert!(self.preload_stream.is_none()); + if self.preload_retry_times >= self.options.max_preload_retry_times { + break; + } + self.preload_retry_times += 1; + tracing::warn!("recreate stream because the connection to remote storage has closed, reason: {:?}", e); + match self + .sstable_store + .preload_blocks(self.sst.value(), idx, self.preload_end_block_idx) + .await + { + Ok(stream) => { + self.preload_stream = stream; + } + Err(e) => { + tracing::error!("failed to recreate stream meet IO error: {:?}", e); + break; + } + } + } + } + } + if !hit_cache { let block = self - .block_fetcher - .get_block(self.sst.value(), idx, &self.sstable_store, &mut self.stats) + .sstable_store + .get( + self.sst.value(), + idx, + self.options.cache_policy, + &mut self.stats, + ) .await?; - let mut block_iter = BlockIterator::new(block); - if let Some(key) = seek_key { - block_iter.seek(key); - } else { - block_iter.seek_to_first(); - } - - self.block_iter = Some(block_iter); - self.cur_idx = idx; + self.block_iter = Some(BlockIterator::new(block)); + }; + let block_iter = self.block_iter.as_mut().unwrap(); + if let Some(key) = seek_key { + block_iter.seek(key); + } else { + block_iter.seek_to_first(); } + self.cur_idx = idx; Ok(()) } } @@ -304,7 +248,7 @@ impl HummockIterator for SstableIterator { fn rewind(&mut self) -> Self::RewindFuture<'_> { async move { - self.init_block_fetcher(0); + self.init_block_prefetch_range(0); self.seek_idx(0, None).await?; Ok(()) } @@ -325,7 +269,7 @@ impl HummockIterator for SstableIterator { ord == Less || ord == Equal }) .saturating_sub(1); // considering the boundary of 0 - self.init_block_fetcher(block_idx); + self.init_block_prefetch_range(block_idx); self.seek_idx(block_idx, Some(key)).await?; if !self.is_valid() { @@ -353,19 +297,24 @@ impl SstableIteratorType for SstableIterator { #[cfg(test)] mod tests { + use std::collections::Bound; + + use bytes::Bytes; use itertools::Itertools; use rand::prelude::*; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; + use risingwave_hummock_sdk::key::{TableKey, UserKey}; use super::*; use crate::assert_bytes_eq; use crate::hummock::iterator::test_utils::mock_sstable_store; use crate::hummock::test_utils::{ - create_small_table_cache, default_builder_opt_for_test, gen_default_test_sstable, - gen_test_sstable, test_key_of, test_value_of, TEST_KEYS_COUNT, + default_builder_opt_for_test, gen_default_test_sstable, gen_test_sstable, test_key_of, + test_value_of, TEST_KEYS_COUNT, }; + use crate::hummock::CachePolicy; async fn inner_test_forward_iterator(sstable_store: SstableStoreRef, handle: TableHolder) { // We should have at least 10 blocks, so that sstable iterator test could cover more code @@ -399,11 +348,9 @@ mod tests { .await; // We should have at least 10 blocks, so that sstable iterator test could cover more code // path. - assert!(sstable.meta.block_metas.len() > 10); + assert!(sstable.value().meta.block_metas.len() > 10); - let cache = create_small_table_cache(); - let handle = cache.insert(0, 0, 1, Box::new(sstable), CachePriority::High); - inner_test_forward_iterator(sstable_store.clone(), handle).await; + inner_test_forward_iterator(sstable_store.clone(), sstable).await; } #[tokio::test] @@ -414,12 +361,9 @@ mod tests { .await; // We should have at least 10 blocks, so that sstable iterator test could cover more code // path. - assert!(sstable.meta.block_metas.len() > 10); - let cache = create_small_table_cache(); - let handle = cache.insert(0, 0, 1, Box::new(sstable), CachePriority::High); - + assert!(sstable.value().meta.block_metas.len() > 10); let mut sstable_iter = SstableIterator::create( - handle, + sstable, sstable_store, Arc::new(SstableIteratorReadOptions::default()), ); @@ -514,16 +458,18 @@ mod tests { ) .await; - let mut stats = StoreLocalStatistic::default(); + let end_key = test_key_of(TEST_KEYS_COUNT / 2); + let uk = UserKey::new( + end_key.user_key.table_id, + TableKey(Bytes::from(end_key.user_key.table_key.0)), + ); let mut sstable_iter = SstableIterator::create( - sstable_store - .sstable(&table.get_sstable_info(), &mut stats) - .await - .unwrap(), + table, sstable_store, Arc::new(SstableIteratorReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), - must_iterated_end_user_key: None, + must_iterated_end_user_key: Some(Bound::Included(uk)), + max_preload_retry_times: 0, }), ); let mut cnt = 0; diff --git a/src/storage/src/hummock/sstable/mod.rs b/src/storage/src/hummock/sstable/mod.rs index 039bf613763b1..7fa5e56007441 100644 --- a/src/storage/src/hummock/sstable/mod.rs +++ b/src/storage/src/hummock/sstable/mod.rs @@ -46,8 +46,6 @@ use risingwave_hummock_sdk::key::{ FullKey, KeyPayloadType, PointRange, TableKey, UserKey, UserKeyRangeRef, }; use risingwave_hummock_sdk::{HummockEpoch, HummockSstableObjectId}; -#[cfg(test)] -use risingwave_pb::hummock::{KeyRange, SstableInfo}; mod delete_range_aggregator; mod filter; @@ -341,24 +339,6 @@ impl Sstable { pub fn estimate_size(&self) -> usize { 8 /* id */ + self.filter_reader.estimate_size() + self.meta.encoded_size() } - - #[cfg(test)] - pub fn get_sstable_info(&self) -> SstableInfo { - SstableInfo { - object_id: self.id, - sst_id: self.id, - key_range: Some(KeyRange { - left: self.meta.smallest_key.clone(), - right: self.meta.largest_key.clone(), - right_exclusive: false, - }), - file_size: self.meta.estimated_size as u64, - meta_offset: self.meta.meta_offset, - total_key_count: self.meta.key_count as u64, - uncompressed_file_size: self.meta.estimated_size as u64, - ..Default::default() - } - } } #[derive(Clone, Default, Debug, Eq, PartialEq)] @@ -594,6 +574,7 @@ impl SstableMeta { pub struct SstableIteratorReadOptions { pub cache_policy: CachePolicy, pub must_iterated_end_user_key: Option>>, + pub max_preload_retry_times: usize, } impl SstableIteratorReadOptions { @@ -601,6 +582,7 @@ impl SstableIteratorReadOptions { Self { cache_policy: read_options.cache_policy, must_iterated_end_user_key: None, + max_preload_retry_times: 0, } } } diff --git a/src/storage/src/hummock/sstable_store.rs b/src/storage/src/hummock/sstable_store.rs index f0b0accb897b3..7a04ba5a2c394 100644 --- a/src/storage/src/hummock/sstable_store.rs +++ b/src/storage/src/hummock/sstable_store.rs @@ -12,12 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. use std::clone::Clone; +use std::collections::VecDeque; use std::future::Future; -use std::sync::atomic::Ordering; +use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; use await_tree::InstrumentAwait; -use bytes::Bytes; +use bytes::{Bytes, BytesMut}; use fail::fail_point; use futures::{future, StreamExt}; use itertools::Itertools; @@ -138,6 +139,8 @@ pub struct SstableStore { meta_file_cache: FileCache>, recent_filter: Option>>, + pending_streaming_loading: Arc, + large_query_memory_usage: usize, } impl SstableStore { @@ -147,6 +150,7 @@ impl SstableStore { block_cache_capacity: usize, meta_cache_capacity: usize, high_priority_ratio: usize, + large_query_memory_usage: usize, data_file_cache: FileCache, meta_file_cache: FileCache>, recent_filter: Option>>, @@ -161,7 +165,6 @@ impl SstableStore { data_file_cache: data_file_cache.clone(), }); let meta_cache_listener = Arc::new(MetaCacheEventListener(meta_file_cache.clone())); - Self { path, store, @@ -182,6 +185,8 @@ impl SstableStore { meta_file_cache, recent_filter, + pending_streaming_loading: Arc::new(AtomicUsize::new(0)), + large_query_memory_usage, } } @@ -201,7 +206,8 @@ impl SstableStore { meta_cache, data_file_cache: FileCache::none(), meta_file_cache: FileCache::none(), - + pending_streaming_loading: Arc::new(AtomicUsize::new(0)), + large_query_memory_usage: block_cache_capacity, recent_filter: None, } } @@ -261,6 +267,49 @@ impl SstableStore { .map_err(HummockError::object_io_error) } + pub async fn preload_blocks( + &self, + sst: &Sstable, + start_index: usize, + mut end_index: usize, + ) -> HummockResult> { + let object_id = sst.id; + if self.block_cache.exists_block(object_id, start_index as u64) { + return Ok(None); + } + let start_offset = sst.meta.block_metas[start_index].offset as usize; + let mut end_offset = start_offset; + for idx in start_index..end_index { + if self.block_cache.exists_block(object_id, idx as u64) { + end_index = idx; + break; + } + end_offset += sst.meta.block_metas[idx].len as usize; + } + let pending_data = self.pending_streaming_loading.load(Ordering::SeqCst); + if end_offset == start_offset || pending_data > self.large_query_memory_usage { + return Ok(None); + } + let data_path = self.get_sst_data_path(object_id); + let memory_usage = std::cmp::min(end_offset - start_offset, self.store.recv_buffer_size()); + self.pending_streaming_loading + .fetch_add(memory_usage, Ordering::SeqCst); + let reader = self + .store + .streaming_read(&data_path, start_offset..end_offset) + .await?; + let block_metas = sst.meta.block_metas[start_index..end_index].to_vec(); + Ok(Some(BatchBlockStream::new( + reader, + self.block_cache.clone(), + object_id, + start_index, + memory_usage, + block_metas, + self.pending_streaming_loading.clone(), + ))) + } + pub async fn get_block_response( &self, sst: &Sstable, @@ -519,10 +568,16 @@ impl SstableStore { .get(block_index) .ok_or_else(HummockError::invalid_block)?; let start_pos = block_meta.offset as usize; + let end_pos = metas[block_index..] + .iter() + .map(|meta| meta.len as usize) + .sum::() + + start_pos; + let range = start_pos..end_pos; Ok(BlockStream::new( store - .streaming_read(&data_path, Some(start_pos)) + .streaming_read(&data_path, range) .await .map_err(HummockError::object_io_error)?, block_index, @@ -883,10 +938,50 @@ impl SstableWriterFactory for StreamingSstableWriterFactory { } } -/// An iterator that reads the blocks of an SST step by step from a given stream of bytes. -pub struct BlockStream { +pub struct BlockStreamReader { /// The stream that provides raw data. byte_stream: MonitoredStreamingReader, +} + +impl BlockStreamReader { + fn new(byte_stream: MonitoredStreamingReader) -> Self { + Self { byte_stream } + } + + async fn next_block( + &mut self, + buf: Bytes, + offset: usize, + read_size: usize, + ) -> HummockResult<(Bytes, Bytes)> { + let mut read_buf = BytesMut::with_capacity(read_size); + let start_pos = if offset < buf.len() { + read_buf.extend_from_slice(&buf[offset..]); + buf.len() - offset + } else { + 0 + }; + let mut rest = read_size - start_pos; + while rest > 0 { + let next_packet = self + .byte_stream + .read_bytes() + .await + .unwrap_or_else(|| Err(ObjectError::internal("read unexpected EOF")))?; + let read_len = std::cmp::min(next_packet.len(), rest); + read_buf.extend_from_slice(&next_packet[..read_len]); + rest -= read_len; + if rest == 0 { + return Ok((read_buf.freeze(), next_packet.slice(read_len..))); + } + } + Ok((read_buf.freeze(), Bytes::default())) + } +} + +/// An iterator that reads the blocks of an SST step by step from a given stream of bytes. +pub struct BlockStream { + reader: BlockStreamReader, /// The index of the next block. Note that `block_idx` is relative to the start index of the /// stream (and is compatible with `block_size_vec`); it is not relative to the corresponding @@ -900,6 +995,10 @@ pub struct BlockStream { /// streaming starts at block 2 of a given SST, then the list does not contain information /// about block 0 and block 1. block_metas: Vec, + + buf: Bytes, + + buff_offset: usize, } impl BlockStream { @@ -922,9 +1021,11 @@ impl BlockStream { let block_index = std::cmp::min(block_index, metas.len()); Self { - byte_stream, + reader: BlockStreamReader::new(byte_stream), block_idx: 0, block_metas: metas[block_index..].to_vec(), + buf: Bytes::default(), + buff_offset: 0, } } @@ -936,27 +1037,26 @@ impl BlockStream { } let block_meta = &self.block_metas[self.block_idx]; - let mut buffer = vec![0; block_meta.len as usize]; fail_point!("stream_read_err", |_| Err(HummockError::object_io_error( ObjectError::internal("stream read error") ))); - - let bytes_read = self - .byte_stream - .read_bytes(&mut buffer[..]) - .await - .map_err(|e| HummockError::object_io_error(ObjectError::internal(e)))?; - - if bytes_read != block_meta.len as usize { - return Err(HummockError::decode_error(ObjectError::internal(format!( - "unexpected number of bytes: expected: {} read: {}", - block_meta.len, bytes_read - )))); - } + let end = self.buff_offset + block_meta.len as usize; + let data = if end > self.buf.len() { + let (current_block, buf) = self + .reader + .next_block(self.buf.clone(), self.buff_offset, block_meta.len as usize) + .await?; + self.buff_offset = 0; + self.buf = buf; + current_block + } else { + let data = self.buf.slice(self.buff_offset..end); + self.buff_offset = end; + data + }; self.block_idx += 1; - - Ok(Some((Bytes::from(buffer), block_meta.clone()))) + Ok(Some((data, block_meta.clone()))) } pub async fn next_block(&mut self) -> HummockResult>> { @@ -970,6 +1070,128 @@ impl BlockStream { } } +/// An iterator that reads the blocks of an SST step by step from a given stream of bytes optimize for batch query. +/// Because all data in batch query shall be put into block-cache, we can not assign a reference of Bytes for each block. +/// We must copy them into new memory. +pub struct BatchBlockStream { + reader: BlockStreamReader, + + block_idx: usize, + + block_metas: Vec, + + buf: Bytes, + + buff_offset: usize, + + blocks: VecDeque, + + cache: BlockCache, + + object_id: HummockSstableObjectId, + + start_block_index: usize, + memory_usage: usize, + /// To avoid high frequently query cost too much memory. + pending_streaming_loading: Arc, +} + +impl BatchBlockStream { + fn new( + byte_stream: MonitoredStreamingReader, + cache: BlockCache, + object_id: HummockSstableObjectId, + start_block_index: usize, + memory_usage: usize, + block_metas: Vec, + pending_streaming_loading: Arc, + ) -> Self { + Self { + reader: BlockStreamReader::new(byte_stream), + block_idx: 0, + block_metas, + buf: Bytes::default(), + buff_offset: 0, + object_id, + memory_usage, + cache, + blocks: VecDeque::default(), + start_block_index, + pending_streaming_loading, + } + } + + /// Reads the next block from the stream and returns it. Returns `None` if there are no blocks + /// left to read. + pub async fn next_block(&mut self) -> HummockResult> { + if self.block_idx >= self.block_metas.len() { + return Ok(None); + } + + let block_meta = &self.block_metas[self.block_idx]; + fail_point!("stream_batch_read_err", |_| Err( + HummockError::object_io_error(ObjectError::internal("stream read error")) + )); + if let Some(block) = self.blocks.pop_front() { + self.block_idx += 1; + return Ok(Some(block)); + } + let (block, buf) = self + .reader + .next_block(self.buf.clone(), self.buff_offset, block_meta.len as usize) + .await?; + self.buf = buf; + self.buff_offset = 0; + let block = Block::decode( + block, + self.block_metas[self.block_idx].uncompressed_size as usize, + )?; + let holder = self.cache.insert( + self.object_id, + (self.start_block_index + self.block_idx) as u64, + Box::new(block), + CachePriority::Low, + ); + let mut block_idx = self.block_idx + 1; + let mut buff_offset = self.buff_offset; + while block_idx < self.block_metas.len() { + let end = buff_offset + self.block_metas[block_idx].len as usize; + if end > self.buf.len() { + break; + } + // copy again to avoid hold a large bytes reference in block-cache. + let block = Block::decode( + Bytes::copy_from_slice(&self.buf[buff_offset..end]), + self.block_metas[block_idx].uncompressed_size as usize, + )?; + let next_holder = self.cache.insert( + self.object_id, + (self.start_block_index + block_idx) as u64, + Box::new(block), + CachePriority::Low, + ); + self.blocks.push_back(next_holder); + buff_offset = end; + block_idx += 1; + } + + self.buff_offset = buff_offset; + self.block_idx += 1; + Ok(Some(holder)) + } + + pub fn next_block_index(&self) -> usize { + self.block_idx + self.start_block_index + } +} + +impl Drop for BatchBlockStream { + fn drop(&mut self) { + self.pending_streaming_loading + .fetch_sub(self.memory_usage, Ordering::SeqCst); + } +} + #[cfg(test)] mod tests { use std::ops::Range; diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 16aa9d474de0a..f58a37c1f7488 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -202,6 +202,7 @@ impl HummockStorage { hummock_version_reader: HummockVersionReader::new( sstable_store, state_store_metrics.clone(), + options.max_preload_io_retry_times, ), _shutdown_guard: Arc::new(HummockStorageShutdownGuard { shutdown_sender: event_tx, diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index c2dba80ee6cfe..0d6a2f7a17a13 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -515,6 +515,7 @@ pub struct HummockVersionReader { /// Statistics state_store_metrics: Arc, + preload_retry_times: usize, } /// use `HummockVersionReader` to reuse `get` and `iter` implement for both `batch_query` and @@ -523,10 +524,12 @@ impl HummockVersionReader { pub fn new( sstable_store: SstableStoreRef, state_store_metrics: Arc, + preload_retry_times: usize, ) -> Self { Self { sstable_store, state_store_metrics, + preload_retry_times, } } @@ -717,7 +720,13 @@ impl HummockVersionReader { .prefix_hint .as_ref() .map(|hint| Sstable::hash_for_bloom_filter(hint, read_options.table_id.table_id())); - + let mut sst_read_options = SstableIteratorReadOptions::from_read_options(&read_options); + if read_options.prefetch_options.preload { + sst_read_options.must_iterated_end_user_key = + Some(user_key_range.1.map(|key| key.cloned())); + sst_read_options.max_preload_retry_times = self.preload_retry_times; + } + let sst_read_options = Arc::new(sst_read_options); for sstable_info in &uncommitted_ssts { let table_holder = self .sstable_store @@ -750,7 +759,7 @@ impl HummockVersionReader { staging_iters.push(HummockIteratorUnion::Second(SstableIterator::new( table_holder, self.sstable_store.clone(), - Arc::new(SstableIteratorReadOptions::from_read_options(&read_options)), + sst_read_options.clone(), ))); } local_stats.staging_sst_iter_count = staging_sst_iter_count; @@ -764,12 +773,6 @@ impl HummockVersionReader { .with_label_values(&[table_id_label]) .start_timer(); - let mut sst_read_options = SstableIteratorReadOptions::from_read_options(&read_options); - if read_options.prefetch_options.exhaust_iter { - sst_read_options.must_iterated_end_user_key = - Some(user_key_range.1.map(|key| key.cloned())); - } - let sst_read_options = Arc::new(sst_read_options); for level in committed.levels(read_options.table_id) { if level.table_infos.is_empty() { continue; diff --git a/src/storage/src/hummock/test_utils.rs b/src/storage/src/hummock/test_utils.rs index 91093804ca5d8..6cf1c66cb9242 100644 --- a/src/storage/src/hummock/test_utils.rs +++ b/src/storage/src/hummock/test_utils.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use bytes::Bytes; use futures::{Stream, TryStreamExt}; use itertools::Itertools; +use risingwave_common::cache::CachePriority; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; use risingwave_common::must_match; @@ -38,7 +39,7 @@ use crate::hummock::value::HummockValue; use crate::hummock::{ create_monotonic_events, BlockedXor16FilterBuilder, CachePolicy, CompactionDeleteRangeIterator, DeleteRangeTombstone, FilterBuilder, LruCache, Sstable, SstableBuilder, SstableBuilderOptions, - SstableStoreRef, SstableWriter, Xor16FilterBuilder, + SstableStoreRef, SstableWriter, TableHolder, Xor16FilterBuilder, }; use crate::monitor::StoreLocalStatistic; use crate::opts::StorageOpts; @@ -274,7 +275,7 @@ pub async fn gen_test_sstable + Clone + Default + Eq>( object_id: HummockSstableObjectId, kv_iter: impl Iterator, HummockValue)>, sstable_store: SstableStoreRef, -) -> Sstable { +) -> TableHolder { let sst_info = gen_test_sstable_impl::<_, Xor16FilterBuilder>( opts, object_id, @@ -284,11 +285,10 @@ pub async fn gen_test_sstable + Clone + Default + Eq>( CachePolicy::NotFill, ) .await; - let table = sstable_store + sstable_store .sstable(&sst_info, &mut StoreLocalStatistic::default()) .await - .unwrap(); - table.value().as_ref().clone() + .unwrap() } /// Generate a test table from the given `kv_iter` and put the kv value to `sstable_store` @@ -316,21 +316,16 @@ pub async fn gen_test_sstable_with_range_tombstone( kv_iter: impl Iterator>, HummockValue>)>, range_tombstones: Vec, sstable_store: SstableStoreRef, -) -> Sstable { - let sst_info = gen_test_sstable_impl::<_, Xor16FilterBuilder>( +) -> SstableInfo { + gen_test_sstable_impl::<_, Xor16FilterBuilder>( opts, object_id, kv_iter, range_tombstones, sstable_store.clone(), - CachePolicy::NotFill, + CachePolicy::Fill(CachePriority::High), ) - .await; - let table = sstable_store - .sstable(&sst_info, &mut StoreLocalStatistic::default()) - .await - .unwrap(); - table.value().as_ref().clone() + .await } /// Generates a user key with table id 0 and the given `table_key` @@ -371,7 +366,7 @@ pub async fn gen_default_test_sstable( opts: SstableBuilderOptions, object_id: HummockSstableObjectId, sstable_store: SstableStoreRef, -) -> Sstable { +) -> TableHolder { gen_test_sstable( opts, object_id, diff --git a/src/storage/src/hummock/validator.rs b/src/storage/src/hummock/validator.rs index e8f6cde87e23d..fe6c402704af2 100644 --- a/src/storage/src/hummock/validator.rs +++ b/src/storage/src/hummock/validator.rs @@ -61,6 +61,7 @@ pub async fn validate_ssts(task: ValidationTask, sstable_store: SstableStoreRef) Arc::new(SstableIteratorReadOptions { cache_policy: CachePolicy::NotFill, must_iterated_end_user_key: None, + max_preload_retry_times: 0, }), ); let mut previous_key: Option>> = None; diff --git a/src/storage/src/opts.rs b/src/storage/src/opts.rs index 2fb02e61512c0..b010d608da119 100644 --- a/src/storage/src/opts.rs +++ b/src/storage/src/opts.rs @@ -49,6 +49,8 @@ pub struct StorageOpts { pub meta_cache_capacity_mb: usize, /// Percent of the ratio of high priority data in block-cache pub high_priority_ratio: usize, + /// max memory usage for large query. + pub large_query_memory_usage_mb: usize, pub disable_remote_compactor: bool, /// Number of tasks shared buffer can upload in parallel. pub share_buffer_upload_concurrency: usize, @@ -126,6 +128,7 @@ pub struct StorageOpts { pub compactor_max_sst_size: u64, /// enable FastCompactorRunner. pub enable_fast_compaction: bool, + pub max_preload_io_retry_times: usize, pub mem_table_spill_threshold: usize, } @@ -157,6 +160,7 @@ impl From<(&RwConfig, &SystemParamsReader, &StorageMemoryConfig)> for StorageOpt write_conflict_detection_enabled: c.storage.write_conflict_detection_enabled, high_priority_ratio: s.high_priority_ratio_in_percent, block_cache_capacity_mb: s.block_cache_capacity_mb, + large_query_memory_usage_mb: s.large_query_memory_usage_mb, meta_cache_capacity_mb: s.meta_cache_capacity_mb, disable_remote_compactor: c.storage.disable_remote_compactor, share_buffer_upload_concurrency: c.storage.share_buffer_upload_concurrency, @@ -234,6 +238,7 @@ impl From<(&RwConfig, &SystemParamsReader, &StorageMemoryConfig)> for StorageOpt .object_store_streaming_upload_timeout_ms, object_store_read_timeout_ms: c.storage.object_store_read_timeout_ms, object_store_upload_timeout_ms: c.storage.object_store_upload_timeout_ms, + max_preload_io_retry_times: c.storage.max_preload_io_retry_times, backup_storage_url: p.backup_storage_url().to_string(), backup_storage_directory: p.backup_storage_directory().to_string(), object_store_recv_buffer_size: c.storage.object_store_recv_buffer_size, diff --git a/src/storage/src/storage_failpoints/test_iterator.rs b/src/storage/src/storage_failpoints/test_iterator.rs index 1d5073e46e023..0b46c2c5a3ea8 100644 --- a/src/storage/src/storage_failpoints/test_iterator.rs +++ b/src/storage/src/storage_failpoints/test_iterator.rs @@ -19,8 +19,9 @@ use std::sync::Arc; use crate::hummock::compactor::{SstableStreamIterator, TaskProgress}; use crate::hummock::iterator::test_utils::{ - gen_iterator_test_sstable_base, iterator_test_bytes_key_of, iterator_test_key_of, - iterator_test_user_key_of, iterator_test_value_of, mock_sstable_store, TEST_KEYS_COUNT, + gen_iterator_test_sstable_base, gen_iterator_test_sstable_info, iterator_test_bytes_key_of, + iterator_test_key_of, iterator_test_user_key_of, iterator_test_value_of, mock_sstable_store, + TEST_KEYS_COUNT, }; use crate::hummock::iterator::{ BackwardConcatIterator, BackwardUserIterator, ConcatIterator, HummockIterator, @@ -41,7 +42,7 @@ async fn test_failpoints_concat_read_err() { fail::cfg("disable_block_cache", "return").unwrap(); let mem_read_err = "mem_read_err"; let sstable_store = mock_sstable_store(); - let table0 = gen_iterator_test_sstable_base( + let table0 = gen_iterator_test_sstable_info( 0, default_builder_opt_for_test(), |x| x * 2, @@ -49,7 +50,7 @@ async fn test_failpoints_concat_read_err() { TEST_KEYS_COUNT, ) .await; - let table1 = gen_iterator_test_sstable_base( + let table1 = gen_iterator_test_sstable_info( 1, default_builder_opt_for_test(), |x| (TEST_KEYS_COUNT + x) * 2, @@ -58,7 +59,7 @@ async fn test_failpoints_concat_read_err() { ) .await; let mut iter = ConcatIterator::new( - vec![table0.get_sstable_info(), table1.get_sstable_info()], + vec![table0, table1], sstable_store, Arc::new(SstableIteratorReadOptions::default()), ); @@ -101,7 +102,7 @@ async fn test_failpoints_backward_concat_read_err() { fail::cfg("disable_block_cache", "return").unwrap(); let mem_read_err = "mem_read_err"; let sstable_store = mock_sstable_store(); - let table0 = gen_iterator_test_sstable_base( + let table0 = gen_iterator_test_sstable_info( 0, default_builder_opt_for_test(), |x| x * 2, @@ -109,7 +110,7 @@ async fn test_failpoints_backward_concat_read_err() { TEST_KEYS_COUNT, ) .await; - let table1 = gen_iterator_test_sstable_base( + let table1 = gen_iterator_test_sstable_info( 1, default_builder_opt_for_test(), |x| (TEST_KEYS_COUNT + x) * 2, @@ -118,7 +119,7 @@ async fn test_failpoints_backward_concat_read_err() { ) .await; let mut iter = BackwardConcatIterator::new( - vec![table1.get_sstable_info(), table0.get_sstable_info()], + vec![table1, table0], sstable_store.clone(), Arc::new(SstableIteratorReadOptions::default()), ); @@ -176,15 +177,9 @@ async fn test_failpoints_merge_invalid_key() { let tables = vec![table0, table1]; let mut mi = UnorderedMergeIteratorInner::new({ let mut iters = vec![]; - for table in &tables { + for table in tables { iters.push(SstableIterator::new( - sstable_store - .sstable( - &table.get_sstable_info(), - &mut StoreLocalStatistic::default(), - ) - .await - .unwrap(), + table, sstable_store.clone(), Arc::new(SstableIteratorReadOptions::default()), )); @@ -230,17 +225,8 @@ async fn test_failpoints_backward_merge_invalid_key() { let tables = vec![table0, table1]; let mut mi = UnorderedMergeIteratorInner::new({ let mut iters = vec![]; - for table in &tables { - iters.push(BackwardSstableIterator::new( - sstable_store - .sstable( - &table.get_sstable_info(), - &mut StoreLocalStatistic::default(), - ) - .await - .unwrap(), - sstable_store.clone(), - )); + for table in tables { + iters.push(BackwardSstableIterator::new(table, sstable_store.clone())); } iters }); @@ -280,21 +266,14 @@ async fn test_failpoints_user_read_err() { 200, ) .await; - let mut stats = StoreLocalStatistic::default(); let iters = vec![ SstableIterator::new( - sstable_store - .sstable(&table0.get_sstable_info(), &mut stats) - .await - .unwrap(), + table0, sstable_store.clone(), Arc::new(SstableIteratorReadOptions::default()), ), SstableIterator::new( - sstable_store - .sstable(&table1.get_sstable_info(), &mut stats) - .await - .unwrap(), + table1, sstable_store.clone(), Arc::new(SstableIteratorReadOptions::default()), ), @@ -347,22 +326,9 @@ async fn test_failpoints_backward_user_read_err() { 200, ) .await; - let mut stats = StoreLocalStatistic::default(); let iters = vec![ - BackwardSstableIterator::new( - sstable_store - .sstable(&table0.get_sstable_info(), &mut stats) - .await - .unwrap(), - sstable_store.clone(), - ), - BackwardSstableIterator::new( - sstable_store - .sstable(&table1.get_sstable_info(), &mut stats) - .await - .unwrap(), - sstable_store.clone(), - ), + BackwardSstableIterator::new(table0, sstable_store.clone()), + BackwardSstableIterator::new(table1, sstable_store.clone()), ]; let mi = UnorderedMergeIteratorInner::new(iters); diff --git a/src/storage/src/storage_failpoints/test_sstable.rs b/src/storage/src/storage_failpoints/test_sstable.rs index d59bbee996108..f6f5aa86e47aa 100644 --- a/src/storage/src/storage_failpoints/test_sstable.rs +++ b/src/storage/src/storage_failpoints/test_sstable.rs @@ -40,7 +40,7 @@ async fn test_failpoints_table_read() { // We should close buffer, so that table iterator must read in object_stores let kv_iter = (0..TEST_KEYS_COUNT).map(|i| (test_key_of(i), HummockValue::put(test_value_of(i)))); - let info = gen_test_sstable( + let table = gen_test_sstable( default_builder_opt_for_test(), 0, kv_iter, @@ -48,12 +48,8 @@ async fn test_failpoints_table_read() { ) .await; - let mut stats = StoreLocalStatistic::default(); let mut sstable_iter = SstableIterator::create( - sstable_store - .sstable(&info.get_sstable_info(), &mut stats) - .await - .unwrap(), + table, sstable_store, Arc::new(SstableIteratorReadOptions::default()), ); diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index f47e5644a388b..b2cc8062f0579 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -120,7 +120,7 @@ impl StateStoreReadExt for S { mut read_options: ReadOptions, ) -> StorageResult> { if limit.is_some() { - read_options.prefetch_options.exhaust_iter = false; + read_options.prefetch_options.preload = false; } let limit = limit.unwrap_or(usize::MAX); self.iter(key_range, epoch, read_options) @@ -274,23 +274,25 @@ pub trait LocalStateStore: StaticSendSync { pub struct PrefetchOptions { /// `exhaust_iter` is set `true` only if the return value of `iter()` will definitely be /// exhausted, i.e., will iterate until end. - pub exhaust_iter: bool, + pub preload: bool, } impl PrefetchOptions { - pub fn new_for_exhaust_iter() -> Self { + pub fn new_for_large_range_scan() -> Self { Self::new_with_exhaust_iter(true) } pub fn new_with_exhaust_iter(exhaust_iter: bool) -> Self { - Self { exhaust_iter } + Self { + preload: exhaust_iter, + } } } impl From for PrefetchOptions { fn from(value: TracedPrefetchOptions) -> Self { Self { - exhaust_iter: value.exhaust_iter, + preload: value.exhaust_iter, } } } @@ -298,7 +300,7 @@ impl From for PrefetchOptions { impl From for TracedPrefetchOptions { fn from(value: PrefetchOptions) -> Self { Self { - exhaust_iter: value.exhaust_iter, + exhaust_iter: value.preload, } } } diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index 734d559e826f7..20957c96ba4ce 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -628,6 +628,7 @@ impl StateStoreImpl { opts.block_cache_capacity_mb * (1 << 20), opts.meta_cache_capacity_mb * (1 << 20), opts.high_priority_ratio, + opts.large_query_memory_usage_mb * (1 << 20), data_file_cache, meta_file_cache, recent_filter, diff --git a/src/stream/src/common/log_store_impl/kv_log_store/reader.rs b/src/stream/src/common/log_store_impl/kv_log_store/reader.rs index d4161da6b581d..c0a759c77746b 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/reader.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/reader.rs @@ -123,7 +123,7 @@ impl LogReader for KvLogStoreReader { (Included(range_start), Excluded(range_end)), MAX_EPOCH, ReadOptions { - prefetch_options: PrefetchOptions::new_for_exhaust_iter(), + prefetch_options: PrefetchOptions::default(), cache_policy: CachePolicy::Fill(CachePriority::Low), table_id, ..Default::default() @@ -234,8 +234,7 @@ impl LogReader for KvLogStoreReader { (Included(range_start), Included(range_end)), MAX_EPOCH, ReadOptions { - prefetch_options: - PrefetchOptions::new_for_exhaust_iter(), + prefetch_options: PrefetchOptions::default(), cache_policy: CachePolicy::Fill(CachePriority::Low), table_id, ..Default::default() diff --git a/src/stream/src/common/table/test_state_table.rs b/src/stream/src/common/table/test_state_table.rs index 7b6d1dce99f21..8587d0c78316b 100644 --- a/src/stream/src/common/table/test_state_table.rs +++ b/src/stream/src/common/table/test_state_table.rs @@ -1015,11 +1015,7 @@ async fn test_state_table_write_chunk() { state_table.write_chunk(chunk); let sub_range: &(Bound, Bound) = &(Unbounded, Unbounded); let rows: Vec<_> = state_table - .iter_with_prefix( - row::empty(), - sub_range, - PrefetchOptions::new_for_exhaust_iter(), - ) + .iter_with_prefix(row::empty(), sub_range, PrefetchOptions::default()) .await .unwrap() .collect::>() @@ -1133,11 +1129,7 @@ async fn test_state_table_write_chunk_visibility() { state_table.write_chunk(chunk); let sub_range: &(Bound, Bound) = &(Unbounded, Unbounded); let rows: Vec<_> = state_table - .iter_with_prefix( - row::empty(), - sub_range, - PrefetchOptions::new_for_exhaust_iter(), - ) + .iter_with_prefix(row::empty(), sub_range, PrefetchOptions::default()) .await .unwrap() .collect::>() @@ -1249,11 +1241,7 @@ async fn test_state_table_write_chunk_value_indices() { state_table.write_chunk(chunk); let sub_range: &(Bound, Bound) = &(Unbounded, Unbounded); let rows: Vec<_> = state_table - .iter_with_prefix( - row::empty(), - sub_range, - PrefetchOptions::new_for_exhaust_iter(), - ) + .iter_with_prefix(row::empty(), sub_range, PrefetchOptions::default()) .await .unwrap() .collect::>() @@ -1535,11 +1523,7 @@ async fn test_state_table_watermark_cache_ignore_null() { state_table.write_chunk(chunk); let sub_range: &(Bound, Bound) = &(Unbounded, Unbounded); let inserted_rows: Vec<_> = state_table - .iter_with_prefix( - row::empty(), - sub_range, - PrefetchOptions::new_for_exhaust_iter(), - ) + .iter_with_prefix(row::empty(), sub_range, PrefetchOptions::default()) .await .unwrap() .collect::>() @@ -1826,11 +1810,7 @@ async fn test_state_table_watermark_cache_refill() { } let sub_range: &(Bound, Bound) = &(Unbounded, Unbounded); let inserted_rows: Vec<_> = state_table - .iter_with_prefix( - row::empty(), - sub_range, - PrefetchOptions::new_for_exhaust_iter(), - ) + .iter_with_prefix(row::empty(), sub_range, PrefetchOptions::default()) .await .unwrap() .collect::>() diff --git a/src/stream/src/executor/aggregation/minput.rs b/src/stream/src/executor/aggregation/minput.rs index 096dfad9cb474..e90058a7ea720 100644 --- a/src/stream/src/executor/aggregation/minput.rs +++ b/src/stream/src/executor/aggregation/minput.rs @@ -178,7 +178,7 @@ impl MaterializedInputState { group_key.map(GroupKey::table_pk), sub_range, PrefetchOptions { - exhaust_iter: cache_filler.capacity().is_none(), + preload: cache_filler.capacity().is_none(), }, ) .await?; diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 99ae7e304e1a9..1d5d2db42404b 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -609,7 +609,7 @@ where row::empty(), range_bounds, ordered, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::new_for_large_range_scan(), ) .await?; diff --git a/src/stream/src/executor/batch_query.rs b/src/stream/src/executor/batch_query.rs index 12f6ff736dc5e..d8bcfbffc8833 100644 --- a/src/stream/src/executor/batch_query.rs +++ b/src/stream/src/executor/batch_query.rs @@ -56,7 +56,7 @@ where .batch_iter( HummockReadEpoch::Committed(epoch), false, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::default(), ) .await?; pin_mut!(iter); diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index 9d98df4cb820e..531aae36bbc55 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -392,7 +392,7 @@ impl DynamicFilterExecutor LookupExecutor { &lookup_row, .., false, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::default(), ) .await? } @@ -395,7 +395,7 @@ impl LookupExecutor { &lookup_row, .., false, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::default(), ) .await? } diff --git a/src/stream/src/executor/managed_state/join/mod.rs b/src/stream/src/executor/managed_state/join/mod.rs index 05f8c1ffde9b3..ca88f829495d5 100644 --- a/src/stream/src/executor/managed_state/join/mod.rs +++ b/src/stream/src/executor/managed_state/join/mod.rs @@ -413,15 +413,14 @@ impl JoinHashMap { if self.need_degree_table { let sub_range: &(Bound, Bound) = &(Bound::Unbounded, Bound::Unbounded); - let table_iter_fut = self.state.table.iter_with_prefix( - &key, - sub_range, - PrefetchOptions::new_for_exhaust_iter(), - ); + let table_iter_fut = + self.state + .table + .iter_with_prefix(&key, sub_range, PrefetchOptions::default()); let degree_table_iter_fut = self.degree_state.table.iter_with_prefix( &key, sub_range, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::default(), ); let (table_iter, degree_table_iter) = @@ -455,7 +454,7 @@ impl JoinHashMap { let table_iter = self .state .table - .iter_with_prefix(&key, sub_range, PrefetchOptions::new_for_exhaust_iter()) + .iter_with_prefix(&key, sub_range, PrefetchOptions::default()) .await?; #[for_await] diff --git a/src/stream/src/executor/over_window/eowc.rs b/src/stream/src/executor/over_window/eowc.rs index 35d87af1a8afe..063b8939c7d5f 100644 --- a/src/stream/src/executor/over_window/eowc.rs +++ b/src/stream/src/executor/over_window/eowc.rs @@ -187,11 +187,7 @@ impl EowcOverWindowExecutor { // Recover states from state table. let table_iter = this .state_table - .iter_with_prefix( - partition_key, - sub_range, - PrefetchOptions::new_for_exhaust_iter(), - ) + .iter_with_prefix(partition_key, sub_range, PrefetchOptions::default()) .await?; #[for_await] diff --git a/src/stream/src/executor/over_window/over_partition.rs b/src/stream/src/executor/over_window/over_partition.rs index 42529a1c80587..b5e9b8019619c 100644 --- a/src/stream/src/executor/over_window/over_partition.rs +++ b/src/stream/src/executor/over_window/over_partition.rs @@ -451,7 +451,7 @@ impl<'a, S: StateStore> OverPartition<'a, S> { .iter_with_prefix( self.this_partition_key, sub_range, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::default(), ) .await?; @@ -565,7 +565,7 @@ impl<'a, S: StateStore> OverPartition<'a, S> { .iter_with_prefix( self.this_partition_key, &table_sub_range, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::default(), ) .await?; @@ -639,7 +639,7 @@ impl<'a, S: StateStore> OverPartition<'a, S> { .iter_with_prefix( self.this_partition_key, &sub_range, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::default(), ) .await?; diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index c45fda5e7bba3..9dd4373dd80ae 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -97,7 +97,7 @@ impl FsFetchExecutor { .iter_with_vnode( vnode, &(Bound::::Unbounded, Bound::::Unbounded), - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::default(), ) .await?; pin_mut!(table_iter); diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index fdacf1100493e..d51d62ebfef06 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -102,11 +102,7 @@ impl SourceStateTableHandler { // all source executor has vnode id zero let iter = self .state_store - .iter_with_vnode( - VirtualNode::ZERO, - &(start, end), - PrefetchOptions::new_for_exhaust_iter(), - ) + .iter_with_vnode(VirtualNode::ZERO, &(start, end), PrefetchOptions::default()) .await?; let mut set = HashSet::new(); diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs index ddfcfd6b8e041..6cb32e8cdadd3 100644 --- a/src/stream/src/executor/temporal_join.rs +++ b/src/stream/src/executor/temporal_join.rs @@ -181,7 +181,7 @@ impl TemporalSide { &pk_prefix, .., false, - PrefetchOptions::new_for_exhaust_iter(), + PrefetchOptions::default(), ) .await?; diff --git a/src/stream/src/executor/top_n/top_n_state.rs b/src/stream/src/executor/top_n/top_n_state.rs index 7214eb91064bc..6885701e39179 100644 --- a/src/stream/src/executor/top_n/top_n_state.rs +++ b/src/stream/src/executor/top_n/top_n_state.rs @@ -128,7 +128,7 @@ impl ManagedTopNState { &group_key, sub_range, PrefetchOptions { - exhaust_iter: cache_size_limit == usize::MAX, + preload: cache_size_limit == usize::MAX, }, ) .await?; @@ -179,7 +179,7 @@ impl ManagedTopNState { &group_key, sub_range, PrefetchOptions { - exhaust_iter: topn_cache.limit == usize::MAX, + preload: topn_cache.limit == usize::MAX, }, ) .await?; diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index db64dc6334c04..27e8ada89f55d 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -210,6 +210,7 @@ async fn compaction_test( storage_memory_config.block_cache_capacity_mb * (1 << 20), storage_memory_config.meta_cache_capacity_mb * (1 << 20), 0, + storage_memory_config.large_query_memory_usage_mb * (1 << 20), FileCache::none(), FileCache::none(), None, @@ -459,7 +460,7 @@ impl NormalState { retention_seconds: None, table_id: self.table_id, read_version_from_backup: false, - prefetch_options: PrefetchOptions::new_for_exhaust_iter(), + prefetch_options: PrefetchOptions::default(), cache_policy: CachePolicy::Fill(CachePriority::High), }, ) @@ -491,7 +492,7 @@ impl CheckState for NormalState { retention_seconds: None, table_id: self.table_id, read_version_from_backup: false, - prefetch_options: PrefetchOptions::new_for_exhaust_iter(), + prefetch_options: PrefetchOptions::default(), cache_policy: CachePolicy::Fill(CachePriority::High), }, ) From 094a0710ea6602ed4e0bfc4469c6e068dc5835f6 Mon Sep 17 00:00:00 2001 From: August Date: Wed, 15 Nov 2023 11:31:01 +0800 Subject: [PATCH 75/77] fix: alter owner automatically for internal table, index and associated source (#13419) --- e2e_test/ddl/alter_owner.slt | 18 +++ src/frontend/src/handler/alter_owner.rs | 38 ++++-- src/meta/src/manager/catalog/mod.rs | 158 ++++++++++++++++++++---- src/meta/src/manager/catalog/user.rs | 9 +- src/meta/src/rpc/ddl_controller.rs | 4 +- 5 files changed, 192 insertions(+), 35 deletions(-) diff --git a/e2e_test/ddl/alter_owner.slt b/e2e_test/ddl/alter_owner.slt index 129443f9abaf2..e5a6773956201 100644 --- a/e2e_test/ddl/alter_owner.slt +++ b/e2e_test/ddl/alter_owner.slt @@ -7,6 +7,9 @@ CREATE USER user1; statement ok CREATE TABLE t (v1 INT primary key, v2 STRUCT>); +statement ok +CREATE INDEX t_idx ON t (v1); + statement ok ALTER TABLE t OWNER TO user1; @@ -25,6 +28,21 @@ WHERE ---- t user1 +query TT +SELECT + pg_class.relname AS rel_name, + pg_roles.rolname AS owner +FROM + pg_class + JOIN pg_namespace ON pg_namespace.oid = pg_class.relnamespace + JOIN pg_roles ON pg_roles.oid = pg_class.relowner +WHERE + pg_namespace.nspname NOT LIKE 'pg_%' + AND pg_namespace.nspname != 'information_schema' + AND pg_class.relname = 't_idx'; +---- +t_idx user1 + statement ok CREATE VIEW v AS ( SELECT * FROM t WHERE v1 = 1); diff --git a/src/frontend/src/handler/alter_owner.rs b/src/frontend/src/handler/alter_owner.rs index d0fc70202528e..4f4ce2550fc98 100644 --- a/src/frontend/src/handler/alter_owner.rs +++ b/src/frontend/src/handler/alter_owner.rs @@ -19,7 +19,7 @@ use risingwave_sqlparser::ast::{Ident, ObjectName}; use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; -use crate::catalog::CatalogError; +use crate::catalog::{CatalogError, OwnedByUserCatalog}; use crate::Binder; pub async fn handle_alter_owner( @@ -36,6 +36,15 @@ pub async fn handle_alter_owner( let user_name = &session.auth_context().user_name; let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name); + let new_owner_name = Binder::resolve_user_name(vec![new_owner_name].into())?; + let owner_id = session + .env() + .user_info_reader() + .read_guard() + .get_user_by_name(&new_owner_name) + .map(|u| u.id) + .ok_or(CatalogError::NotFound("user", new_owner_name))?; + let object = { let catalog_reader = session.env().catalog_reader().read_guard(); match stmt_type { @@ -43,49 +52,58 @@ pub async fn handle_alter_owner( let (table, schema_name) = catalog_reader.get_table_by_name(db_name, schema_path, &real_obj_name)?; session.check_privilege_for_drop_alter(schema_name, &**table)?; + if table.owner() == owner_id { + return Ok(RwPgResponse::empty_result(stmt_type)); + } Object::TableId(table.id.table_id) } StatementType::ALTER_VIEW => { let (view, schema_name) = catalog_reader.get_view_by_name(db_name, schema_path, &real_obj_name)?; session.check_privilege_for_drop_alter(schema_name, &**view)?; + if view.owner() == owner_id { + return Ok(RwPgResponse::empty_result(stmt_type)); + } Object::ViewId(view.id) } StatementType::ALTER_SOURCE => { let (source, schema_name) = catalog_reader.get_source_by_name(db_name, schema_path, &real_obj_name)?; session.check_privilege_for_drop_alter(schema_name, &**source)?; + if source.owner() == owner_id { + return Ok(RwPgResponse::empty_result(stmt_type)); + } Object::SourceId(source.id) } StatementType::ALTER_SINK => { let (sink, schema_name) = catalog_reader.get_sink_by_name(db_name, schema_path, &real_obj_name)?; session.check_privilege_for_drop_alter(schema_name, &**sink)?; + if sink.owner() == owner_id { + return Ok(RwPgResponse::empty_result(stmt_type)); + } Object::SinkId(sink.id.sink_id) } StatementType::ALTER_DATABASE => { let database = catalog_reader.get_database_by_name(&obj_name.real_value())?; session.check_privilege_for_drop_alter_db_schema(database)?; + if database.owner() == owner_id { + return Ok(RwPgResponse::empty_result(stmt_type)); + } Object::DatabaseId(database.id()) } StatementType::ALTER_SCHEMA => { let schema = catalog_reader.get_schema_by_name(db_name, &obj_name.real_value())?; session.check_privilege_for_drop_alter_db_schema(schema)?; + if schema.owner() == owner_id { + return Ok(RwPgResponse::empty_result(stmt_type)); + } Object::SchemaId(schema.id()) } _ => unreachable!(), } }; - let new_owner_name = Binder::resolve_user_name(vec![new_owner_name].into())?; - let owner_id = session - .env() - .user_info_reader() - .read_guard() - .get_user_by_name(&new_owner_name) - .map(|u| u.id) - .ok_or(CatalogError::NotFound("user", new_owner_name))?; - let catalog_writer = session.catalog_writer()?; catalog_writer.alter_owner(object, owner_id).await?; diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 65daa69def417..5fdf3c4a61c71 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -45,7 +45,9 @@ use risingwave_pb::user::{GrantPrivilege, UserInfo}; use tokio::sync::{Mutex, MutexGuard}; use user::*; -use crate::manager::{IdCategory, MetaSrvEnv, NotificationVersion, StreamingJob}; +use crate::manager::{ + IdCategory, MetaSrvEnv, NotificationVersion, StreamingJob, IGNORED_NOTIFICATION_VERSION, +}; use crate::model::{BTreeMapTransaction, MetadataModel, TableFragments, ValTransaction}; use crate::storage::Transaction; use crate::{MetaError, MetaResult}; @@ -1768,6 +1770,7 @@ impl CatalogManager { pub async fn alter_owner( &self, + fragment_manager: FragmentManagerRef, object: alter_owner_request::Object, owner_id: UserId, ) -> MetaResult { @@ -1776,26 +1779,99 @@ impl CatalogManager { let user_core = &mut core.user; let notify_info; - let old_owner_id; match object { alter_owner_request::Object::TableId(table_id) => { database_core.ensure_table_id(table_id)?; let mut tables = BTreeMapTransaction::new(&mut database_core.tables); - let mut table = tables.get_mut(table_id).unwrap(); - old_owner_id = table.owner; - table.owner = owner_id; - notify_info = Info::RelationGroup(RelationGroup { - relations: vec![Relation { + let mut sources = BTreeMapTransaction::new(&mut database_core.sources); + let mut indexes = BTreeMapTransaction::new(&mut database_core.indexes); + + let table = tables.tree_ref().get(&table_id).unwrap(); + let old_owner_id = table.owner; + if old_owner_id == owner_id { + return Ok(IGNORED_NOTIFICATION_VERSION); + } + // associated source id. + let to_update_source_id = if let Some( + OptionalAssociatedSourceId::AssociatedSourceId(associated_source_id), + ) = &table.optional_associated_source_id + { + Some(*associated_source_id) + } else { + None + }; + + let mut to_update_table_ids = vec![table_id]; + let mut to_update_internal_table_ids = vec![]; + + // indexes and index tables. + let (to_update_index_ids, index_table_ids): (Vec<_>, Vec<_>) = indexes + .tree_ref() + .iter() + .filter(|(_, index)| index.primary_table_id == table_id) + .map(|(index_id, index)| (*index_id, index.index_table_id)) + .unzip(); + to_update_table_ids.extend(index_table_ids); + + // internal tables. + for id in &to_update_table_ids { + let table_fragment = fragment_manager + .select_table_fragments_by_table_id(&(id.into())) + .await?; + to_update_internal_table_ids.extend(table_fragment.internal_table_ids()); + } + + let mut relations = vec![]; + // update owner. + for id in &to_update_table_ids { + let mut table = tables.get_mut(*id).unwrap(); + assert_eq!(old_owner_id, table.owner); + table.owner = owner_id; + relations.push(Relation { relation_info: Some(RelationInfo::Table(table.clone())), - }], - }); - commit_meta!(self, tables)?; + }); + } + for index_id in &to_update_index_ids { + let mut index = indexes.get_mut(*index_id).unwrap(); + assert_eq!(old_owner_id, index.owner); + index.owner = owner_id; + relations.push(Relation { + relation_info: Some(RelationInfo::Index(index.clone())), + }); + } + if let Some(associated_source_id) = &to_update_source_id { + let mut source = sources.get_mut(*associated_source_id).unwrap(); + assert_eq!(old_owner_id, source.owner); + source.owner = owner_id; + relations.push(Relation { + relation_info: Some(RelationInfo::Source(source.clone())), + }); + } + for internal_table_id in to_update_internal_table_ids { + let mut table = tables.get_mut(internal_table_id).unwrap(); + assert_eq!(old_owner_id, table.owner); + table.owner = owner_id; + relations.push(Relation { + relation_info: Some(RelationInfo::Table(table.clone())), + }); + } + + commit_meta!(self, tables, indexes, sources)?; + let count = to_update_table_ids.len() + + to_update_index_ids.len() + + to_update_source_id.map_or(0, |_| 1); + user_core.decrease_ref_count(old_owner_id, count); + user_core.increase_ref_count(owner_id, count); + notify_info = Info::RelationGroup(RelationGroup { relations }); } alter_owner_request::Object::ViewId(view_id) => { database_core.ensure_view_id(view_id)?; let mut views = BTreeMapTransaction::new(&mut database_core.views); let mut view = views.get_mut(view_id).unwrap(); - old_owner_id = view.owner; + let old_owner_id = view.owner; + if old_owner_id == owner_id { + return Ok(IGNORED_NOTIFICATION_VERSION); + } view.owner = owner_id; notify_info = Info::RelationGroup(RelationGroup { relations: vec![Relation { @@ -1803,12 +1879,17 @@ impl CatalogManager { }], }); commit_meta!(self, views)?; + user_core.increase_ref(owner_id); + user_core.decrease_ref(old_owner_id); } alter_owner_request::Object::SourceId(source_id) => { database_core.ensure_source_id(source_id)?; let mut sources = BTreeMapTransaction::new(&mut database_core.sources); let mut source = sources.get_mut(source_id).unwrap(); - old_owner_id = source.owner; + let old_owner_id = source.owner; + if old_owner_id == owner_id { + return Ok(IGNORED_NOTIFICATION_VERSION); + } source.owner = owner_id; notify_info = Info::RelationGroup(RelationGroup { relations: vec![Relation { @@ -1816,41 +1897,72 @@ impl CatalogManager { }], }); commit_meta!(self, sources)?; + user_core.increase_ref(owner_id); + user_core.decrease_ref(old_owner_id); } alter_owner_request::Object::SinkId(sink_id) => { database_core.ensure_sink_id(sink_id)?; let mut sinks = BTreeMapTransaction::new(&mut database_core.sinks); + let mut tables = BTreeMapTransaction::new(&mut database_core.tables); let mut sink = sinks.get_mut(sink_id).unwrap(); - old_owner_id = sink.owner; + let old_owner_id = sink.owner; + if old_owner_id == owner_id { + return Ok(IGNORED_NOTIFICATION_VERSION); + } sink.owner = owner_id; - notify_info = Info::RelationGroup(RelationGroup { - relations: vec![Relation { - relation_info: Some(RelationInfo::Sink(sink.clone())), - }], - }); - commit_meta!(self, sinks)?; + + let mut relations = vec![Relation { + relation_info: Some(RelationInfo::Sink(sink.clone())), + }]; + + // internal tables + let internal_table_ids = fragment_manager + .select_table_fragments_by_table_id(&(sink_id.into())) + .await? + .internal_table_ids(); + for id in internal_table_ids { + let mut table = tables.get_mut(id).unwrap(); + assert_eq!(old_owner_id, table.owner); + table.owner = owner_id; + relations.push(Relation { + relation_info: Some(RelationInfo::Table(table.clone())), + }); + } + + notify_info = Info::RelationGroup(RelationGroup { relations }); + commit_meta!(self, sinks, tables)?; + user_core.increase_ref(owner_id); + user_core.decrease_ref(old_owner_id); } alter_owner_request::Object::DatabaseId(database_id) => { database_core.ensure_database_id(database_id)?; let mut databases = BTreeMapTransaction::new(&mut database_core.databases); let mut database = databases.get_mut(database_id).unwrap(); - old_owner_id = database.owner; + let old_owner_id = database.owner; + if old_owner_id == owner_id { + return Ok(IGNORED_NOTIFICATION_VERSION); + } database.owner = owner_id; notify_info = Info::Database(database.clone()); commit_meta!(self, databases)?; + user_core.increase_ref(owner_id); + user_core.decrease_ref(old_owner_id); } alter_owner_request::Object::SchemaId(schema_id) => { database_core.ensure_schema_id(schema_id)?; let mut schemas = BTreeMapTransaction::new(&mut database_core.schemas); let mut schema = schemas.get_mut(schema_id).unwrap(); - old_owner_id = schema.owner; + let old_owner_id = schema.owner; + if old_owner_id == owner_id { + return Ok(IGNORED_NOTIFICATION_VERSION); + } schema.owner = owner_id; notify_info = Info::Schema(schema.clone()); commit_meta!(self, schemas)?; + user_core.increase_ref(owner_id); + user_core.decrease_ref(old_owner_id); } }; - user_core.increase_ref(owner_id); - user_core.decrease_ref(old_owner_id); let version = self.notify_frontend(Operation::Update, notify_info).await; diff --git a/src/meta/src/manager/catalog/user.rs b/src/meta/src/manager/catalog/user.rs index ff68ef21cbcb0..3a6d110b7c301 100644 --- a/src/meta/src/manager/catalog/user.rs +++ b/src/meta/src/manager/catalog/user.rs @@ -16,6 +16,7 @@ use std::collections::hash_map::Entry; use std::collections::{BTreeMap, HashMap, HashSet}; use anyhow::anyhow; +use risingwave_pb::catalog::table::TableType; use risingwave_pb::user::UserInfo; use super::database::DatabaseManager; @@ -52,7 +53,13 @@ impl UserManager { .chain(database.sources.values().map(|source| source.owner)) .chain(database.sinks.values().map(|sink| sink.owner)) .chain(database.indexes.values().map(|index| index.owner)) - .chain(database.tables.values().map(|table| table.owner)) + .chain( + database + .tables + .values() + .filter(|table| table.table_type() != TableType::Internal) + .map(|table| table.owner), + ) .chain(database.views.values().map(|view| view.owner)) .for_each(|owner_id| user_manager.increase_ref(owner_id)); diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 0cd3af7385184..bf171951da8a5 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -1137,7 +1137,9 @@ impl DdlController { object: Object, owner_id: UserId, ) -> MetaResult { - self.catalog_manager.alter_owner(object, owner_id).await + self.catalog_manager + .alter_owner(self.fragment_manager.clone(), object, owner_id) + .await } pub async fn wait(&self) -> MetaResult<()> { From 28fc3a870adc9c1695e6450660acf8ab0c8bbc12 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 15 Nov 2023 12:14:37 +0800 Subject: [PATCH 76/77] doc: clarify skip-ci use (#13431) --- .github/pull_request_template.md | 1 + docs/developer-guide.md | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 328915a649e3c..17d5bbfb8af84 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -20,6 +20,7 @@ Please explain **IN DETAIL** what the changes are in this PR and why they are ne - [ ] I have written necessary rustdoc comments - [ ] I have added necessary unit tests and integration tests +- [ ] I have added test labels as necessary. See [details](https://github.com/risingwavelabs/risingwave/blob/main/docs/developer-guide.md#ci-labels-guide). - [ ] I have added fuzzing tests or opened an issue to track them. (Optional, recommended for new SQL features #7934). - [ ] My PR contains breaking changes. (If it deprecates some features, please create a tracking issue to remove them in the future). - [ ] All checks passed in `./risedev check` (or alias, `./risedev c`) diff --git a/docs/developer-guide.md b/docs/developer-guide.md index 2782db7e29e62..f01a8414eebbf 100644 --- a/docs/developer-guide.md +++ b/docs/developer-guide.md @@ -527,10 +527,12 @@ Instructions about submitting PRs are included in the [contribution guidelines]( ## CI Labels Guide -- `ci/skip-ci` + `[ci/run-xxx ...]` : Run specific steps indicated by `ci/run-xxx` in your **DRAFT PR.** +- `[ci/run-xxx ...]`: Run additional steps indicated by `ci/run-xxx` in your PR. +- `ci/skip-ci` + `[ci/run-xxx ...]` : Skip steps except for those indicated by `ci/run-xxx` in your **DRAFT PR.** - `ci/run-main-cron`: Run full `main-cron`. - `ci/run-main-cron` + `ci/main-cron/skip-ci` + `[ci/run-xxx …]` : Run specific steps indicated by `ci/run-xxx` from the `main-cron` workflow, in your PR. Can use to verify some `main-cron` fix works as expected. +- To reference `[ci/run-xxx ...]` labels, you may look at steps from `pull-request.yml` and `main-cron.yml`. - **Be sure to add all the dependencies.** For example to run `e2e-test` for `main-cron` in your pull request: From 88977cb7053624fe9d02e489a77a58ea1277ab3b Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Wed, 15 Nov 2023 13:36:25 +0800 Subject: [PATCH 77/77] fix(test): pulsar test timeout in main-cron (#13407) --- ci/scripts/e2e-pulsar-sink-test.sh | 1 + src/connector/src/sink/pulsar.rs | 16 +++++++++++++++- src/connector/src/sink/writer.rs | 9 ++++++--- 3 files changed, 22 insertions(+), 4 deletions(-) diff --git a/ci/scripts/e2e-pulsar-sink-test.sh b/ci/scripts/e2e-pulsar-sink-test.sh index a2a0edb550f33..ee8848832f940 100755 --- a/ci/scripts/e2e-pulsar-sink-test.sh +++ b/ci/scripts/e2e-pulsar-sink-test.sh @@ -30,6 +30,7 @@ MAX_RETRY=20 while [[ $HTTP_CODE -ne 200 && MAX_RETRY -gt 0 ]] do HTTP_CODE=$(curl --connect-timeout 2 -s -o /dev/null -w ''%{http_code}'' http://pulsar:8080/admin/v2/clusters) + echo Got HTTP Code: $HTTP_CODE ((MAX_RETRY--)) sleep 5 done diff --git a/src/connector/src/sink/pulsar.rs b/src/connector/src/sink/pulsar.rs index f536890c932cd..aef116733b6de 100644 --- a/src/connector/src/sink/pulsar.rs +++ b/src/connector/src/sink/pulsar.rs @@ -264,7 +264,10 @@ impl<'w> PulsarPayloadWriter<'w> { let mut success_flag = false; let mut connection_err = None; - for _ in 0..self.config.max_retry_num { + for retry_num in 0..self.config.max_retry_num { + if retry_num > 0 { + tracing::warn!("Failed to send message, at retry no. {retry_num}"); + } match self.producer.send(message.clone()).await { // If the message is sent successfully, // a SendFuture holding the message receipt @@ -351,4 +354,15 @@ impl AsyncTruncateSinkWriter for PulsarSinkWriter { Ok(()) }) } + + async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { + if is_checkpoint { + self.producer + .send_batch() + .map_err(pulsar_to_sink_err) + .await?; + } + + Ok(()) + } } diff --git a/src/connector/src/sink/writer.rs b/src/connector/src/sink/writer.rs index 64261bb42ab48..50f4cbb0ed1b9 100644 --- a/src/connector/src/sink/writer.rs +++ b/src/connector/src/sink/writer.rs @@ -67,6 +67,10 @@ pub trait AsyncTruncateSinkWriter: Send + 'static { chunk: StreamChunk, add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, ) -> impl Future> + Send + 'a; + + fn barrier(&mut self, _is_checkpoint: bool) -> impl Future> + Send + '_ { + async { Ok(()) } + } } /// A free-form sink that may output in multiple formats and encodings. Examples include kafka, @@ -261,9 +265,8 @@ impl LogSinker for AsyncTruncateLogSinkerOf { let add_future = self.future_manager.start_write_chunk(epoch, chunk_id); self.writer.write_chunk(chunk, add_future).await?; } - LogStoreReadItem::Barrier { - is_checkpoint: _is_checkpoint, - } => { + LogStoreReadItem::Barrier { is_checkpoint } => { + self.writer.barrier(is_checkpoint).await?; self.future_manager.add_barrier(epoch); } LogStoreReadItem::UpdateVnodeBitmap(_) => {}