From 4894d44f8d6f7d6a2901be2b3c80ce54b2c954f8 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Thu, 23 Feb 2023 16:37:01 +0800 Subject: [PATCH] Squashed commit of the following: MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit commit f2199feddd757512b5d06d5a443f66ca3cf7ae1c Author: stonepage <40830455+st1page@users.noreply.github.com> Date: Thu Feb 23 16:27:42 2023 +0800 feat(explain): add conflict behavior in explain materialize operator (#8138) as title Approved-By: BugenZhao commit 52a39fd2b550ea9a63c35c2a22224b8dbc5733e0 Author: jon-chuang <9093549+jon-chuang@users.noreply.github.com> Date: Thu Feb 23 16:09:34 2023 +0800 feat(stream): `ErrorSuppressor` for user compute errors (#8132) `ErrorSuppressor` for user compute errors Approved-By: fuyufjh Co-Authored-By: jon-chuang Co-Authored-By: jon-chuang <9093549+jon-chuang@users.noreply.github.com> commit f5f8f83005a94516cc1426dc65a09b38809d1ea9 Author: idx0-dev <124041366+idx0-dev@users.noreply.github.com> Date: Thu Feb 23 15:43:13 2023 +0800 feat: kafka-upsert with json,avro format (#8111) To support `upsert-kafka` in a manner similar to [how Flink does](https://nightlies.apache.org/flink/flink-docs-release-1.16/docs/connectors/table/upsert-kafka/), the key field of a Kafka message is used to indicate the values of the primary key column. If the value field of the message is not empty, the row will be inserted or updated. If the value field is empty, the row will be deleted. This behavior is not tied to any specific row format. A Kafka connector with the `upsert` property enabled will produce `UpsertMessage`s encoded in bytes, instead of raw Kafka message values, as `SourceMessage`s. The row formats prefixed with `UPSERT_` are aware that `SourceMessage`s contain not only the Kafka message value field but also the key field as primary columns, and will behave as expected. Approved-By: waruto210 Co-Authored-By: idx0-dev <124041366+idx0-dev@users.noreply.github.com> Co-Authored-By: waruto commit dd7fc13f2510db133afd07a767f1cd87af54fa4d Author: Yuanxin Cao <60498509+xx01cyx@users.noreply.github.com> Date: Thu Feb 23 15:24:02 2023 +0800 feat(streaming): enable dml executor to pause and resume on scaling (#8110) - Enable dml executor to pause and resume on scaling. - A little refactor on `StreamReaderWithPause` (previously named `SourceReaderStream`): - Make the left arm accept general message types instead of barriers only. - Introduce non-biased `StreamReaderWithPause`. Fixes #8056 Approved-By: st1page Approved-By: waruto210 commit ba92df425cb5eb091adc66656bfe6330108ab826 Author: waruto Date: Thu Feb 23 15:04:46 2023 +0800 fix: remove message name parameter for avro schema (#8124) I checked the code for the version where this parameter was first introduced and found that it was never used, it was probably just copied from `ProtobufSchema`. It is strange to let the user specify the message name for a avro schema, so we should remove it. Approved-By: tabVersion Approved-By: hzxa21 commit 5c050efdf92411fa19bdee6a8bf7735c9dbed42b Author: Bugen Zhao Date: Thu Feb 23 14:42:36 2023 +0800 feat: fill correct table version ID for DML (#8120) This PR implements versioning for DML statements. - Frontend: use the correct version ID extracted from the catalog to fill the fields of batch DML plan node protos. - Connector: do sanity check on the table schema when registering with the same version. - Meta: fill the `version` field of streaming DML plan node proto when visiting the fragment graph. Approved-By: chenzl25 Approved-By: st1page Approved-By: xx01cyx commit 88dc35e44554152a88366f6535d7a60b37a4bf9f Author: jon-chuang <9093549+jon-chuang@users.noreply.github.com> Date: Thu Feb 23 08:09:40 2023 +0800 feat(frontend): apply `SessionTimezone` and `ConstEvalRewriter` expr rewriters to during `gen_{batch,stream}_plan` (#7761) apply `SessionTimezone` and `ConstEvalRewriter` expr rewriters to during `gen_{batch,stream}_plan` Notes: - wait for https://github.com/risingwavelabs/risingwave/pull/7757 to be merged - wait for https://github.com/risingwavelabs/risingwave/pull/7777 to be merged - wait for https://github.com/risingwavelabs/risingwave/pull/7786 to be merged Approved-By: ice1000 Approved-By: st1page Co-Authored-By: jon-chuang Co-Authored-By: jon-chuang <9093549+jon-chuang@users.noreply.github.com> commit 7cadc39dffda32a01587a08913591cf9666fe84e Author: Zhidong Guo <52783948+Gun9niR@users.noreply.github.com> Date: Thu Feb 23 03:19:36 2023 +0800 feat(meta): mutable checkpoint frequency (#8010) As title. Use `LocalNotification` to asyncly notify other components on the meta node of the latest params. Approved-By: BugenZhao Co-Authored-By: Gun9niR Co-Authored-By: Zhidong Guo <52783948+Gun9niR@users.noreply.github.com> commit 3a598fbc8aa9e1118dfd12760046d714d988ed02 Author: Wallace Date: Wed Feb 22 20:04:37 2023 +0800 fix(storage): fix calculate incorrect memory usage of sstable meta (#8126) close https://github.com/risingwavelabs/risingwave/issues/8125 Approved-By: soundOfDestiny Approved-By: Li0k commit c60a5dbf443baa1e2d6c59da3778dd8456be741d Author: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed Feb 22 19:13:44 2023 +0800 fix(ci): make java-binding-e2e release test depends on build-release (#8127) In the release CI yaml, there is no item named `build`, which is different to the PR CI. Therefore, the CI in PR works, but fails on release CI. This PR fix it. Approved-By: xxchan commit 8619b1119cf07f34c6a0ed1d0792773c439f1d98 Author: Wallace Date: Wed Feb 22 18:21:53 2023 +0800 fix(storage): fix skip delete range in uncommitted files (#8009) Approved-By: Li0k commit 864fb46af50e4bbc3617854d97a8eaa5f6fae0a6 Author: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed Feb 22 17:44:56 2023 +0800 feat(expr): access `jsonb` object field and array element (#8023) Adds the following expressions: * `jsonb_object_field(jsonb, varchar) -> jsonb` * `jsonb_array_element(jsonb, int) -> jsonb` * `jsonb_object_field_text(jsonb, varchar) -> varchar` * `jsonb_array_element_text(jsonb, int) -> varchar` * `jsonb_typeof(jsonb) -> varchar` * `jsonb_array_length(jsonb) -> int` The first two are actually operator `->` in PostgreSQL, and the two in the middle are operator `->>` in PostgreSQL. But our parser does not support parsing this syntax yet. The optimization of constant rhs will be added in a followup. Approved-By: BugenZhao Co-Authored-By: Xiangjin Co-Authored-By: Xiangjin commit 88cb075c523bab60183d5a09b3b3e1a4bd37cd89 Author: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Wed Feb 22 16:56:07 2023 +0800 refactor: add CastError for cast function (#8090) add the specified error for cast function. refer more detail: #8074 Approved-By: xiangjinwu Approved-By: BugenZhao commit 05e7a0e724aab01c2a505da8574d1567958e9132 Author: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Wed Feb 22 16:35:07 2023 +0800 refactor(storage): OpenDAL backend use batch delete (#8054) Approved-By: Li0k Co-Authored-By: congyi <15605187270@163.com> Co-Authored-By: congyi wang <58715567+wcy-fdu@users.noreply.github.com> commit 014eb0954b5d0024be937212279ccbb7af341dda Author: August Date: Wed Feb 22 15:59:04 2023 +0800 feat(meta): export metrics of meta count/role info (#8057) Export metrics of meta count and role infos to grafana. Approved-By: shanicky commit 8dff6203efc1f512abfb71f9d02a99493c32b6f0 Author: Bugen Zhao Date: Wed Feb 22 15:35:18 2023 +0800 feat(streaming): support output indices in dispatchers (#8094) This PR adds support for output indices in each dispatcher. Here are the motivations: - For multiple MVs on an upstream MV, it's possible that each of them requires different columns of the upstreams. Currently, we do this projection in the downstream `Chain` node. However, if we allow creating mview on remote compute nodes (like spot instances), directly pruning the unused columns in upstream will decrease the remote shuffle cost as described in https://github.com/risingwavelabs/risingwave/issues/4529. - For adding columns in schema change, there should be a layer that erases the schema change from `Materialize` to the downstream. By introducing the output indices in dispatchers, we can make the existing downstream MV receive chunks with the same schema and work correctly. For new downstream MVs after schema change, the new dispatcher will be able to output all columns. (#6903) Note that the optimization mentioned in Motivation 1 is not implemented in this PR. Currently, we just always output all columns in every dispatcher. Approved-By: fuyufjh Approved-By: chenzl25 Approved-By: xxchan commit 8e499c72740da56a63aab8e53b9f6fc071bdbe80 Author: Shanicky Chen Date: Wed Feb 22 15:15:54 2023 +0800 fix: Update EtcdElectionClient keep_alive/observe behavior (#8058) Approved-By: yezizp2012 commit 6c5f68f9df0b90155cf2be3000cc8f08860000fe Author: Yuanxin Cao <60498509+xx01cyx@users.noreply.github.com> Date: Wed Feb 22 14:33:02 2023 +0800 feat(sink): prune out hidden columns on sink (#8099) - Prune out hidden columns on sink - Reject upsert sink without pk after pruning - Refine `StreamSink` explain format Approved-By: tabVersion Approved-By: st1page commit 229a3c7b531cd5a98e3274ef0cf5a312804e8cbe Author: jon-chuang <9093549+jon-chuang@users.noreply.github.com> Date: Wed Feb 22 12:52:39 2023 +0800 feat(stream): `source_error_count` reporting to prometheus (#7877) Source stream error reporting to prometheus ![image](https://user-images.githubusercontent.com/9093549/218451821-fd1cbccf-e28b-42f2-a777-1fa88cccf6a4.png) Approved-By: fuyufjh Co-Authored-By: jon-chuang Co-Authored-By: jon-chuang <9093549+jon-chuang@users.noreply.github.com> commit 8a242febdd916cb7727f992368fda25a728367fc Author: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Wed Feb 22 12:30:00 2023 +0800 chore(ci): add log for flaky meta backup test (#8109) Add logs to troubleshoot flaky test in #7850 Approved-By: Li0k commit fe9c5c51632ababcba2f09f6ec8d6c57bd4270a2 Author: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed Feb 22 12:09:08 2023 +0800 test(java-binding): add ci for java-binding (#7942) As titled. Added a CI case for java-binding that launches a RisingWave cluster, inserts data to the table, and runs the java binding demo to read data from the table. The previous cargo make script to run the demo is split for better reuse of script code. Approved-By: hzxa21 Approved-By: Gun9niR Co-Authored-By: William Wen Co-Authored-By: William Wen <44139337+wenym1@users.noreply.github.com> commit e16e26df637e25e076cb77cc82d2ec49e66b6f8d Author: Dylan Date: Wed Feb 22 11:42:42 2023 +0800 feat(frontend): describe stmt shows index ordering (#8073) - As title. Approved-By: yezizp2012 Approved-By: cyliu0 Co-Authored-By: Dylan Chen Co-Authored-By: Dylan commit b429e9c2966a88c47f71ea0e05f4dc3589801448 Author: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed Feb 22 10:44:18 2023 +0800 fix(common): `ListArray::from_protobuf` expects wrong cardinality of inner array (#8091) The internal of `ListArray` stores its data flattened. For example, `values (array[1]), (array[]::int[]), (null), (array[2, 3]);` stores an inner `I32Array` with `[1, 2, 3]`, along with offset array `[0, 1, 1, 1, 3]` and null bitmap `TTFT`. The cardinality of this inner array is not the length of outer array, but the last element of offset array. Fixes #8082 Approved-By: kwannoel Co-Authored-By: Xiangjin Co-Authored-By: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> commit 8bc69d4647ee51043bc47457abe1b63104707a29 Author: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed Feb 22 08:18:57 2023 +0800 fix(batch): enforce order for `LogicalValues` created by empty `LogicalScan` (#8079) - Fix #8067. Approved-By: chenzl25 Approved-By: jon-chuang Co-Authored-By: Noel Kwan Co-Authored-By: Noel Kwan <47273164+kwannoel@users.noreply.github.com> commit 4b6e09393344902dcfaaf8eb1bb2f3873c809b67 Author: jon-chuang <9093549+jon-chuang@users.noreply.github.com> Date: Wed Feb 22 01:29:36 2023 +0800 feat(frontend): add `trace!` to optimizer trace. (#8092) add `trace!` to optimizer trace. Easier debugging for when a frontend optimization step goes wrong. Approved-By: kwannoel Approved-By: fuyufjh Co-Authored-By: jon-chuang Co-Authored-By: jon-chuang <9093549+jon-chuang@users.noreply.github.com> commit f11c53bc70e44dff075fc3ec561d0f8991302388 Author: Bugen Zhao Date: Wed Feb 22 00:39:59 2023 +0800 fix: iterate vnode bitmap with `iter_vnodes` (#8083) This is to avoid confusing `usize` with `VirtualNode` when iterating the vnode bitmap as much as possible. For example, we've found that the `DeleteRange` is incorrect caused by calling `usize::to_be_bytes` by mistake for vnode prefix. Approved-By: soundOfDestiny Approved-By: TennyZhuang Approved-By: hzxa21 Co-Authored-By: Bugen Zhao Co-Authored-By: TennyZhuang commit 48351604b148c2f14e79742e80c7b341c3bacad9 Author: Li0k Date: Tue Feb 21 21:31:29 2023 +0800 chore(storage): remove state store v1 (#8102) remove unused code state_store_v1 and local_version_manager Approved-By: wenym1 commit 7a0316b57c898096b6549ea2feeddc1de0daff77 Author: Runji Wang Date: Tue Feb 21 20:46:29 2023 +0800 feat(udf): minimal Python UDF SDK (#7943) This PR designs a minimal SDK for Python UDFs. Now you can define a function in Python like this: ```python from risingwave.udf import udf, UdfServer @udf(input_types=['INT', 'INT'], result_type='INT') def gcd(x: int, y: int) -> int: while y != 0: (x, y) = (y, x % y) return x if __name__ == '__main__': server = UdfServer() server.add_function(gcd) server.serve() ``` This PR also fixes the problem when functions have no input arguments. Approved-By: xxchan Approved-By: BugenZhao commit f11bb625ed57ebb909323ee398dd96557f430ca1 Author: xxchan Date: Tue Feb 21 13:16:18 2023 +0100 doc: add e2e_test/generated/README.md (#8104) 💦 Approved-By: richardchien commit 4b374c35a4c5c21a956da4634ed3211ea2e502c1 Author: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue Feb 21 20:13:45 2023 +0800 test(sink): update the link to download spark tgz for iceberg test (#8095) Fix https://github.com/risingwavelabs/risingwave/issues/8093 Approved-By: xxchan Approved-By: Li0k Approved-By: jon-chuang Co-Authored-By: William Wen Co-Authored-By: William Wen <44139337+wenym1@users.noreply.github.com> commit 672aad56a70cfa0bbd48aa07556cc6d20a19fdc5 Author: Li0k Date: Tue Feb 21 19:56:23 2023 +0800 feat(storage): Introduce TtlReclaimSelector and refactor the trigger logic of Scheduler (#7937) Its part of https://github.com/risingwavelabs/risingwave/issues/6918 Improve and introduce TtlReclaimSelector, and introduce TtlReclaimTrigger to periodically initiate compaction against ttl for LastLevel to ensure that data can be reclaimed in a timely manner. As more triggers are introduced, try to refactor the Scheduler's Trigger logic to ensure the maintainability of the code. - Stream to simplify the code of the Scheduler trigger - Replace the last_index policy with key_range to ensure that the compaction runs correctly Approved-By: zwang28 Approved-By: Little-Wallace Co-Authored-By: Li0k Co-Authored-By: Runji Wang Signed-off-by: Richard Chien --- .gitignore | 6 +- Cargo.lock | 336 +++++---- Cargo.toml | 8 + ci/Dockerfile | 5 +- ci/build-ci-image.sh | 2 +- ci/docker-compose.yml | 14 +- ci/scripts/build.sh | 4 +- ci/scripts/e2e-iceberg-sink-test.sh | 2 +- ci/scripts/e2e-sink-test.sh | 1 + ci/scripts/java-binding-test.sh | 55 ++ ci/scripts/run-e2e-test.sh | 6 + ci/workflows/main.yml | 13 + ci/workflows/pull-request.yml | 13 + dashboard/proto/gen/catalog.ts | 5 + dashboard/proto/gen/common.ts | 6 + dashboard/proto/gen/expr.ts | 28 +- dashboard/proto/gen/hummock.ts | 10 + dashboard/proto/gen/plan_common.ts | 12 + dashboard/proto/gen/stream_plan.ts | 54 +- e2e_test/batch/explain.slt | 4 +- e2e_test/batch/functions/now.slt.part | 9 +- e2e_test/batch/types/array_ty.slt.part | 14 + e2e_test/batch/types/jsonb.slt.part | 36 + e2e_test/ddl/function.slt | 27 - e2e_test/ddl/invalid_operation.slt | 2 +- e2e_test/ddl/table.slt | 4 +- e2e_test/generated/README.md | 11 + e2e_test/sink/append_only_sink.slt | 53 ++ e2e_test/sink/blackhole_sink.slt | 2 +- e2e_test/source/basic/kafka.slt | 9 +- e2e_test/streaming/join.slt | 37 + e2e_test/udf/python.slt | 49 ++ e2e_test/udf/test.py | 29 + grafana/risingwave-dashboard.dashboard.py | 25 +- grafana/risingwave-dashboard.json | 2 +- proto/catalog.proto | 1 + proto/common.proto | 1 + proto/expr.proto | 11 +- proto/hummock.proto | 2 + proto/plan_common.proto | 2 + proto/stream_plan.proto | 11 +- risedev.yml | 1 - src/batch/src/executor/source.rs | 2 +- src/common/Cargo.toml | 5 +- src/common/src/array/column_proto_readers.rs | 8 +- src/common/src/array/decimal_array.rs | 2 +- src/common/src/array/jsonb_array.rs | 40 ++ src/common/src/array/list_array.rs | 19 +- src/common/src/config.rs | 24 +- src/common/src/error.rs | 40 ++ src/common/src/hash/consistent_hash/bitmap.rs | 34 + .../src/hash/consistent_hash/mapping.rs | 5 +- src/common/src/hash/consistent_hash/mod.rs | 1 + src/common/src/hash/mod.rs | 1 + .../{system_param.rs => system_param/mod.rs} | 2 + src/common/src/system_param/reader.rs | 82 +++ src/connector/Cargo.toml | 2 +- src/connector/src/common.rs | 10 + src/connector/src/macros.rs | 8 +- src/connector/src/parser/avro/parser.rs | 174 ++++- .../src/parser/avro/schema_resolver.rs | 42 +- .../src/parser/canal/simd_json_parser.rs | 14 +- src/connector/src/parser/csv_parser.rs | 14 +- .../src/parser/debezium/avro_parser.rs | 16 +- .../src/parser/debezium/simd_json_parser.rs | 25 +- src/connector/src/parser/json_parser.rs | 162 ++++- .../src/parser/maxwell/simd_json_parser.rs | 12 +- src/connector/src/parser/mod.rs | 64 +- src/connector/src/parser/protobuf/parser.rs | 12 +- src/connector/src/source/base.rs | 51 +- .../src/source/filesystem/s3/source/reader.rs | 12 +- src/connector/src/source/kafka/mod.rs | 6 + .../src/source/kafka/source/message.rs | 20 + .../src/source/kafka/source/reader.rs | 13 +- src/connector/src/source/monitor/metrics.rs | 16 + src/expr/Cargo.toml | 4 +- src/expr/src/expr/build_expr_from_prost.rs | 8 +- src/expr/src/expr/expr_array_to_string.rs | 8 +- src/expr/src/expr/expr_binary_nonnull.rs | 39 +- src/expr/src/expr/expr_jsonb_access.rs | 222 ++++++ src/expr/src/expr/expr_udf.rs | 19 +- src/expr/src/expr/expr_unary.rs | 13 + src/expr/src/expr/mod.rs | 1 + src/expr/src/sig/func.rs | 7 + src/expr/src/vector_op/jsonb_info.rs | 36 + src/expr/src/vector_op/mod.rs | 1 + .../planner_test/tests/testdata/agg.yaml | 72 +- .../tests/testdata/append_only.yaml | 8 +- .../planner_test/tests/testdata/array.yaml | 14 +- .../tests/testdata/basic_query.yaml | 24 +- .../planner_test/tests/testdata/cast.yaml | 4 +- .../tests/testdata/column_pruning.yaml | 2 +- .../testdata/common_table_expressions.yaml | 6 +- .../tests/testdata/delta_join.yaml | 6 +- .../tests/testdata/distribution_derive.yaml | 72 +- .../tests/testdata/dynamic_filter.yaml | 12 +- .../planner_test/tests/testdata/explain.yaml | 14 +- .../planner_test/tests/testdata/expr.yaml | 95 +-- .../tests/testdata/index_selection.yaml | 8 +- .../planner_test/tests/testdata/insert.yaml | 12 +- .../planner_test/tests/testdata/join.yaml | 18 +- .../tests/testdata/mv_column_name.yaml | 6 +- .../planner_test/tests/testdata/mv_on_mv.yaml | 2 +- .../planner_test/tests/testdata/nexmark.yaml | 221 +++--- .../tests/testdata/nexmark_source.yaml | 246 +++---- .../planner_test/tests/testdata/order_by.yaml | 39 +- .../tests/testdata/over_window_function.yaml | 10 +- .../tests/testdata/pk_derive.yaml | 8 +- .../tests/testdata/predicate_pushdown.yaml | 10 +- .../tests/testdata/project_set.yaml | 24 +- .../tests/testdata/range_scan.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 26 +- .../tests/testdata/shared_views.yaml | 6 +- .../tests/testdata/stream_dist_agg.yaml | 156 ++--- .../tests/testdata/struct_query.yaml | 2 +- .../tests/testdata/sysinfo_funcs.yaml | 4 +- .../tests/testdata/temporal_filter.yaml | 20 +- .../tests/testdata/time_window.yaml | 16 +- .../planner_test/tests/testdata/tpch.yaml | 384 +++++------ .../planner_test/tests/testdata/union.yaml | 12 +- .../tests/testdata/watermark.yaml | 4 +- src/frontend/src/binder/delete.rs | 7 +- src/frontend/src/binder/expr/function.rs | 7 + src/frontend/src/binder/expr/mod.rs | 3 +- src/frontend/src/binder/expr/value.rs | 2 +- src/frontend/src/binder/insert.rs | 9 +- src/frontend/src/binder/update.rs | 7 +- src/frontend/src/catalog/table_catalog.rs | 5 + src/frontend/src/expr/function_call.rs | 70 +- src/frontend/src/expr/mod.rs | 20 +- src/frontend/src/expr/table_function.rs | 6 +- src/frontend/src/expr/type_inference/cast.rs | 6 +- src/frontend/src/expr/type_inference/func.rs | 8 +- src/frontend/src/expr/window_function.rs | 4 +- src/frontend/src/handler/create_sink.rs | 2 +- src/frontend/src/handler/create_source.rs | 94 +++ src/frontend/src/handler/create_table_as.rs | 6 +- src/frontend/src/handler/describe.rs | 20 +- src/frontend/src/handler/drop_sink.rs | 2 +- src/frontend/src/meta_client.rs | 3 +- src/frontend/src/optimizer/mod.rs | 70 +- .../src/optimizer/optimizer_context.rs | 13 +- .../src/optimizer/plan_node/batch_delete.rs | 3 +- .../src/optimizer/plan_node/batch_filter.rs | 6 +- .../src/optimizer/plan_node/batch_hash_agg.rs | 2 +- .../optimizer/plan_node/batch_hash_join.rs | 7 +- .../src/optimizer/plan_node/batch_insert.rs | 3 +- .../optimizer/plan_node/batch_lookup_join.rs | 14 +- .../plan_node/batch_nested_loop_join.rs | 7 +- .../src/optimizer/plan_node/batch_project.rs | 8 +- .../optimizer/plan_node/batch_simple_agg.rs | 4 +- .../src/optimizer/plan_node/batch_sort_agg.rs | 9 +- .../src/optimizer/plan_node/batch_update.rs | 11 +- .../src/optimizer/plan_node/batch_values.rs | 11 +- .../src/optimizer/plan_node/derive.rs | 4 +- .../src/optimizer/plan_node/generic/agg.rs | 7 +- .../src/optimizer/plan_node/logical_delete.rs | 26 +- .../src/optimizer/plan_node/logical_insert.rs | 12 +- .../src/optimizer/plan_node/logical_scan.rs | 18 +- .../src/optimizer/plan_node/logical_source.rs | 3 +- .../src/optimizer/plan_node/logical_update.rs | 20 +- .../src/optimizer/plan_node/stream.rs | 40 +- .../optimizer/plan_node/stream_delta_join.rs | 7 +- .../src/optimizer/plan_node/stream_dml.rs | 5 +- .../optimizer/plan_node/stream_exchange.rs | 3 +- .../src/optimizer/plan_node/stream_filter.rs | 8 +- .../plan_node/stream_global_simple_agg.rs | 3 +- .../optimizer/plan_node/stream_hash_agg.rs | 3 +- .../optimizer/plan_node/stream_hash_join.rs | 7 +- .../plan_node/stream_local_simple_agg.rs | 3 +- .../optimizer/plan_node/stream_materialize.rs | 7 + .../src/optimizer/plan_node/stream_project.rs | 7 +- .../src/optimizer/plan_node/stream_share.rs | 14 +- .../src/optimizer/plan_node/stream_sink.rs | 38 +- .../optimizer/plan_node/stream_table_scan.rs | 3 +- src/frontend/src/planner/delete.rs | 11 +- src/frontend/src/planner/insert.rs | 1 + src/frontend/src/planner/update.rs | 1 + src/frontend/src/session.rs | 1 - src/frontend/src/stream_fragmenter/mod.rs | 3 +- .../stream_fragmenter/rewrite/delta_join.rs | 43 +- src/frontend/src/test_utils.rs | 2 +- .../src/utils/column_index_mapping.rs | 2 +- .../src/utils/stream_graph_formatter.rs | 2 +- src/java_binding/make-java-binding.toml | 55 +- src/java_binding/run_demo.sh | 22 +- src/meta/src/backup_restore/restore.rs | 2 +- src/meta/src/barrier/mod.rs | 21 +- src/meta/src/barrier/schedule.rs | 13 +- .../src/hummock/compaction/level_selector.rs | 25 +- src/meta/src/hummock/compaction/mod.rs | 12 +- .../picker/manual_compaction_picker.rs | 6 +- .../picker/space_reclaim_compaction_picker.rs | 291 +++++++- .../picker/ttl_reclaim_compaction_picker.rs | 596 +++++++++++++++- .../src/hummock/compaction_schedule_policy.rs | 2 + src/meta/src/hummock/compaction_scheduler.rs | 344 +++++++--- src/meta/src/hummock/manager/mod.rs | 33 +- src/meta/src/hummock/manager/worker.rs | 1 + .../src/hummock/mock_hummock_meta_client.rs | 2 +- src/meta/src/hummock/mod.rs | 1 - src/meta/src/hummock/test_utils.rs | 2 + src/meta/src/lib.rs | 16 +- src/meta/src/manager/catalog/database.rs | 9 +- src/meta/src/manager/catalog/mod.rs | 4 +- src/meta/src/manager/cluster.rs | 35 +- src/meta/src/manager/env.rs | 6 +- src/meta/src/manager/notification.rs | 2 + src/meta/src/manager/streaming_job.rs | 8 +- src/meta/src/manager/system_param/mod.rs | 8 + src/meta/src/rpc/election_client.rs | 45 +- src/meta/src/rpc/metrics.rs | 66 +- src/meta/src/rpc/server.rs | 22 +- src/meta/src/stream/stream_graph/actor.rs | 36 +- src/meta/src/stream/stream_graph/fragment.rs | 14 +- src/meta/src/stream/test_fragmenter.rs | 9 +- src/object_store/Cargo.toml | 2 +- .../opendal_engine/opendal_object_store.rs | 39 +- src/object_store/src/object/s3.rs | 35 - src/rpc_client/src/lib.rs | 2 +- src/rpc_client/src/meta_client.rs | 68 +- src/source/benches/json_parser.rs | 2 +- src/source/src/dml_manager.rs | 35 +- src/source/src/source_desc.rs | 2 + src/source/src/table.rs | 6 +- src/sqlparser/src/ast/statement.rs | 35 +- src/sqlparser/src/keywords.rs | 2 + .../backup/integration_tests/common.sh | 30 +- .../backup/integration_tests/run_all.sh | 2 + .../integration_tests/test_query_backup.sh | 57 +- .../src/hummock_read_version_tests.rs | 4 + src/storage/hummock_test/src/lib.rs | 2 - .../src/local_version_manager_tests.rs | 3 - .../hummock_test/src/snapshot_tests.rs | 179 +---- .../hummock_test/src/state_store_tests.rs | 58 +- src/storage/hummock_test/src/test_utils.rs | 41 +- .../src/hummock/event_handler/uploader.rs | 2 + .../local_version/local_version_manager.rs | 384 ----------- src/storage/src/hummock/local_version/mod.rs | 1 - src/storage/src/hummock/mod.rs | 164 +---- src/storage/src/hummock/sstable/builder.rs | 31 +- src/storage/src/hummock/sstable/mod.rs | 4 +- src/storage/src/hummock/sstable/xor_filter.rs | 4 + src/storage/src/hummock/sstable_store.rs | 2 +- src/storage/src/hummock/state_store_v1.rs | 638 ------------------ src/storage/src/hummock/store/version.rs | 11 +- src/storage/src/hummock/test_utils.rs | 4 + src/storage/src/opts.rs | 4 +- src/storage/src/store_impl.rs | 107 +-- .../src/table/batch_table/storage_table.rs | 10 +- src/stream/Cargo.toml | 1 + src/stream/src/common/table/state_table.rs | 4 +- src/stream/src/executor/actor.rs | 39 +- src/stream/src/executor/dispatch.rs | 56 +- src/stream/src/executor/dml.rs | 50 +- src/stream/src/executor/dynamic_filter.rs | 10 +- src/stream/src/executor/exchange/output.rs | 25 +- src/stream/src/executor/mod.rs | 1 + .../src/executor/monitor/streaming_stats.rs | 12 +- src/stream/src/executor/sort.rs | 6 +- src/stream/src/executor/sort_buffer.rs | 9 +- .../src/executor/source/fs_source_executor.rs | 103 +-- src/stream/src/executor/source/mod.rs | 19 +- .../src/executor/source/source_executor.rs | 104 +-- .../{source/reader.rs => stream_reader.rs} | 117 ++-- src/stream/src/executor/watermark_filter.rs | 6 +- src/stream/src/task/stream_manager.rs | 1 + .../src/delete_range_runner.rs | 13 +- src/udf/Cargo.toml | 6 +- src/udf/README.md | 2 +- src/udf/arrow_flight.py | 55 -- src/udf/examples/client.rs | 43 +- src/udf/python/example.py | 27 + src/udf/python/risingwave/__init__.py | 0 src/udf/python/risingwave/udf.py | 168 +++++ src/udf/src/lib.rs | 14 +- src/workspace-hack/Cargo.toml | 4 +- 276 files changed, 5373 insertions(+), 3738 deletions(-) create mode 100755 ci/scripts/java-binding-test.sh delete mode 100644 e2e_test/ddl/function.slt create mode 100644 e2e_test/generated/README.md create mode 100644 e2e_test/sink/append_only_sink.slt create mode 100644 e2e_test/udf/python.slt create mode 100644 e2e_test/udf/test.py create mode 100644 src/common/src/hash/consistent_hash/bitmap.rs rename src/common/src/{system_param.rs => system_param/mod.rs} (99%) create mode 100644 src/common/src/system_param/reader.rs create mode 100644 src/expr/src/expr/expr_jsonb_access.rs create mode 100644 src/expr/src/vector_op/jsonb_info.rs delete mode 100644 src/storage/src/hummock/local_version/local_version_manager.rs delete mode 100644 src/storage/src/hummock/state_store_v1.rs rename src/stream/src/executor/{source/reader.rs => stream_reader.rs} (56%) delete mode 100644 src/udf/arrow_flight.py create mode 100644 src/udf/python/example.py create mode 100644 src/udf/python/risingwave/__init__.py create mode 100644 src/udf/python/risingwave/udf.py diff --git a/.gitignore b/.gitignore index 907131a7ac187..3ebf11628f38c 100644 --- a/.gitignore +++ b/.gitignore @@ -23,6 +23,9 @@ cmake-build-debug/ *.app build/ +# Python +*.pyc + # Golang go/bin/ @@ -58,6 +61,7 @@ ft.txt src/tests/regress/output/* src/tests/sync_point/slt/e2e_test # generated e2e tests -e2e_test/generated +e2e_test/generated/* +!e2e_test/generated/README.md # generated scale tests by `cargo nextest` scale-test.tar.zst diff --git a/Cargo.lock b/Cargo.lock index 5b87392d090e6..808cf0b33963e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -134,9 +134,8 @@ checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" [[package]] name = "arrow-array" -version = "31.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1e6e839764618a911cc460a58ebee5ad3d42bc12d9a5e96a29b7cc296303aa1" +version = "33.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=9a6c516#9a6c516f6e5c5411489a65af2e53dba041a26025" dependencies = [ "ahash 0.8.3", "arrow-buffer", @@ -150,9 +149,8 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "31.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03a21d232b1bc1190a3fdd2f9c1e39b7cd41235e95a0d44dd4f522bc5f495748" +version = "33.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=9a6c516#9a6c516f6e5c5411489a65af2e53dba041a26025" dependencies = [ "half 2.2.1", "num", @@ -160,9 +158,8 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "31.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83dcdb1436cac574f1c1b30fda91c53c467534337bef4064bbd4ea2d6fbc6e04" +version = "33.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=9a6c516#9a6c516f6e5c5411489a65af2e53dba041a26025" dependencies = [ "arrow-array", "arrow-buffer", @@ -176,9 +173,8 @@ dependencies = [ [[package]] name = "arrow-data" -version = "31.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14e3e69c9fd98357eeeab4aa0f626ecf7ecf663e68e8fc04eac87c424a414477" +version = "33.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=9a6c516#9a6c516f6e5c5411489a65af2e53dba041a26025" dependencies = [ "arrow-buffer", "arrow-schema", @@ -188,9 +184,8 @@ dependencies = [ [[package]] name = "arrow-flight" -version = "31.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd3ce08d31a1a24497bcf144029f8475539984aa50e41585e01b2057cf3dbb21" +version = "33.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=9a6c516#9a6c516f6e5c5411489a65af2e53dba041a26025" dependencies = [ "arrow-array", "arrow-buffer", @@ -211,9 +206,8 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "31.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64cac2706acbd796965b6eaf0da30204fe44aacf70273f8cb3c9b7d7f3d4c190" +version = "33.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=9a6c516#9a6c516f6e5c5411489a65af2e53dba041a26025" dependencies = [ "arrow-array", "arrow-buffer", @@ -225,15 +219,13 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "31.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73ca49d010b27e2d73f70c1d1f90c1b378550ed0f4ad379c4dea0c997d97d723" +version = "33.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=9a6c516#9a6c516f6e5c5411489a65af2e53dba041a26025" [[package]] name = "arrow-select" -version = "31.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "976cbaeb1a85c09eea81f3f9c149c758630ff422ed0238624c5c3f4704b6a53c" +version = "33.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=9a6c516#9a6c516f6e5c5411489a65af2e53dba041a26025" dependencies = [ "arrow-array", "arrow-buffer", @@ -380,19 +372,20 @@ dependencies = [ [[package]] name = "async-stream" -version = "0.3.3" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dad5c83079eae9969be7fadefe640a1c566901f05ff91ab221de4b6f68d9507e" +checksum = "ad445822218ce64be7a341abfb0b1ea43b5c23aa83902542a4542e78309d8e5e" dependencies = [ "async-stream-impl", "futures-core", + "pin-project-lite", ] [[package]] name = "async-stream-impl" -version = "0.3.3" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10f203db73a71dfa2fb6dd22763990fa26f3d2625a6da2da900d23b87d26be27" +checksum = "e4655ae1a7b0cdf149156f780c5bf3f1352bc53cbd9e0a361a7ef7b22947e965" dependencies = [ "proc-macro2", "quote", @@ -500,9 +493,9 @@ checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" [[package]] name = "autotools" -version = "0.2.5" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8138adefca3e5d2e73bfba83bd6eeaf904b26a7ac1b4a19892cfe16cc7e1701" +checksum = "aef8da1805e028a172334c3b680f93e71126f2327622faef2ec3d893c0a4ad77" dependencies = [ "cc", ] @@ -528,7 +521,7 @@ dependencies = [ "http", "hyper", "ring", - "time 0.3.18", + "time 0.3.17", "tokio", "tower", "tracing", @@ -691,7 +684,7 @@ dependencies = [ "percent-encoding", "regex", "ring", - "time 0.3.18", + "time 0.3.17", "tracing", ] @@ -829,7 +822,7 @@ dependencies = [ "itoa", "num-integer", "ryu", - "time 0.3.18", + "time 0.3.17", ] [[package]] @@ -859,9 +852,9 @@ dependencies = [ [[package]] name = "axum" -version = "0.6.6" +version = "0.6.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e246206a63c9830e118d12c894f56a82033da1a2361f5544deeee3df85c99d9" +checksum = "2fb79c228270dcf2426e74864cabc94babb5dbab01a4314e702d2f16540e1591" dependencies = [ "async-trait", "axum-core", @@ -907,18 +900,6 @@ dependencies = [ "tower-service", ] -[[package]] -name = "backon" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6cd1a59bc091e593ee9ed62df4e4a07115e00a0e0a52fd7e0e04540773939b80" -dependencies = [ - "futures", - "pin-project", - "rand 0.8.5", - "tokio", -] - [[package]] name = "backon" version = "0.4.0" @@ -971,6 +952,12 @@ version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a4a4ddaa51a5bc52a6948f74c06d20aaaddb71924eab79b8c97a8c556e942d6a" +[[package]] +name = "base64ct" +version = "1.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b645a089122eccb6111b4f81cbc1a49f5900ac4666bb93ac027feaecf15607bf" + [[package]] name = "bcc" version = "0.0.33" @@ -1000,25 +987,6 @@ dependencies = [ "serde", ] -[[package]] -name = "bincode" -version = "2.0.0-rc.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7bb50c5a2ef4b9b1e7ae73e3a73b52ea24b20312d629f9c4df28260b7ad2c3c4" -dependencies = [ - "bincode_derive", - "serde", -] - -[[package]] -name = "bincode_derive" -version = "2.0.0-rc.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a45a23389446d2dd25dc8e73a7a3b3c43522b630cac068927f0649d43d719d2" -dependencies = [ - "virtue", -] - [[package]] name = "bit-vec" version = "0.6.3" @@ -1111,9 +1079,9 @@ dependencies = [ [[package]] name = "bstr" -version = "1.2.0" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b7f0778972c64420fdedc63f09919c8a88bda7b25135357fd25a5d9f3257e832" +checksum = "5ffdb39cb703212f3c11973452c2861b972f757b021158f3516ba10f2fa8b2c1" dependencies = [ "memchr", "serde", @@ -1215,9 +1183,9 @@ dependencies = [ [[package]] name = "camino" -version = "1.1.2" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c77df041dc383319cc661b428b6961a005db4d6808d5e12536931b1ca9556055" +checksum = "6031a462f977dd38968b6f23378356512feeace69cef817e1a4475108093cec3" dependencies = [ "serde", ] @@ -1531,6 +1499,12 @@ dependencies = [ "tracing-subscriber", ] +[[package]] +name = "const-oid" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cec318a675afcb6a1ea1d4340e2d377e56e47c266f28043ceccbf4412ddfdd3b" + [[package]] name = "const-random" version = "0.1.15" @@ -1830,9 +1804,9 @@ dependencies = [ [[package]] name = "curl-sys" -version = "0.4.59+curl-7.86.0" +version = "0.4.60+curl-7.88.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6cfce34829f448b08f55b7db6d0009e23e2e86a34e8c2b366269bf5799b4a407" +checksum = "717abe2cb465a5da6ce06617388a3980c9a2844196734bec8ccb8e575250f13f" dependencies = [ "cc", "libc", @@ -1978,6 +1952,17 @@ dependencies = [ "uuid", ] +[[package]] +name = "der" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1a467a65c5e759bce6e65eaf91cc29f466cdc57cb65777bd646872a8a1fd4de" +dependencies = [ + "const-oid", + "pem-rfc7468", + "zeroize", +] + [[package]] name = "derivative" version = "2.2.0" @@ -2025,6 +2010,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8168378f4e5023e7218c89c891c0fd8ecdb5e5e4f18cb78f38cf245dd021e76f" dependencies = [ "block-buffer", + "const-oid", "crypto-common", "subtle", ] @@ -2084,7 +2070,7 @@ dependencies = [ "rust_decimal", "serde", "thiserror", - "time 0.3.18", + "time 0.3.17", ] [[package]] @@ -2291,12 +2277,12 @@ checksum = "cda653ca797810c02f7ca4b804b40b8b95ae046eb989d356bce17919a8c25499" [[package]] name = "flatbuffers" -version = "22.9.29" +version = "23.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ce016b9901aef3579617931fbb2df8fc9a9f7cb95a16eb8acc8148209bb9e70" +checksum = "77f5399c2c9c50ae9418e522842ad362f61ee48b346ac106807bd355a8a7c619" dependencies = [ "bitflags", - "thiserror", + "rustc_version", ] [[package]] @@ -2837,9 +2823,9 @@ dependencies = [ [[package]] name = "http" -version = "0.2.8" +version = "0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75f43d41e26995c17e71ee126451dd3941010b0514a81a9d11f3b341debc2399" +checksum = "bd6effc99afb63425aff9b05836f029929e345a6148a14b7ecd5ab67af944482" dependencies = [ "bytes", "fnv", @@ -3263,6 +3249,9 @@ name = "lazy_static" version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +dependencies = [ + "spin 0.5.2", +] [[package]] name = "lexical-core" @@ -3336,9 +3325,9 @@ checksum = "201de327520df007757c1f0adce6e827fe8562fbc28bfd9c15571c66ca1f5f79" [[package]] name = "libflate" -version = "1.2.0" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05605ab2bce11bcfc0e9c635ff29ef8b2ea83f29be257ee7d730cac3ee373093" +checksum = "97822bf791bd4d5b403713886a5fbe8bf49520fe78e323b0dc480ca1a03e50b0" dependencies = [ "adler32", "crc32fast", @@ -3347,9 +3336,9 @@ dependencies = [ [[package]] name = "libflate_lz77" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39a734c0493409afcd49deee13c006a04e3586b9761a03543c6272c9c51f2f5a" +checksum = "a52d3a8bfc85f250440e4424db7d857e241a3aebbbe301f3eb606ab15c39acbf" dependencies = [ "rle-decode-fast", ] @@ -3504,7 +3493,7 @@ dependencies = [ "ahash 0.7.6", "async-channel", "async-task", - "bincode 1.3.3", + "bincode", "bytes", "downcast-rs", "futures-util", @@ -3696,9 +3685,9 @@ dependencies = [ [[package]] name = "memmap2" -version = "0.5.8" +version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b182332558b18d807c4ce1ca8ca983b34c3ee32765e47b3f0f69b90355cc1dc" +checksum = "2af2c65375e552a67fe3829ca63e8a7c27a378a62824594f43b2851d682b5ec2" dependencies = [ "libc", ] @@ -3892,27 +3881,28 @@ dependencies = [ [[package]] name = "nix" -version = "0.24.3" +version = "0.25.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa52e972a9a719cecb6864fb88568781eb706bac2cd1d4f04a648542dbf78069" +checksum = "f346ff70e7dbfd675fe90590b92d59ef2de15a8779ae305ebcbfd3f0caf59be4" dependencies = [ + "autocfg", "bitflags", "cfg-if", "libc", + "memoffset 0.6.5", + "pin-utils", ] [[package]] name = "nix" -version = "0.25.1" +version = "0.26.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f346ff70e7dbfd675fe90590b92d59ef2de15a8779ae305ebcbfd3f0caf59be4" +checksum = "bfdda3d196821d6af13126e40375cdf7da646a96114af134d5f417a9a1dc8e1a" dependencies = [ - "autocfg", "bitflags", "cfg-if", "libc", - "memoffset 0.6.5", - "pin-utils", + "static_assertions", ] [[package]] @@ -3978,6 +3968,23 @@ dependencies = [ "num-traits", ] +[[package]] +name = "num-bigint-dig" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2399c9463abc5f909349d8aa9ba080e0b88b3ce2885389b60b993f39b1a56905" +dependencies = [ + "byteorder", + "lazy_static", + "libm", + "num-integer", + "num-iter", + "num-traits", + "rand 0.8.5", + "smallvec", + "zeroize", +] + [[package]] name = "num-complex" version = "0.4.3" @@ -4063,18 +4070,18 @@ dependencies = [ [[package]] name = "num_enum" -version = "0.5.9" +version = "0.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d829733185c1ca374f17e52b762f24f535ec625d2cc1f070e34c8a9068f341b" +checksum = "3e0072973714303aa6e3631c7e8e777970cf4bdd25dc4932e41031027b8bcc4e" dependencies = [ "num_enum_derive", ] [[package]] name = "num_enum_derive" -version = "0.5.9" +version = "0.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2be1598bf1c313dcdd12092e3f1920f463462525a21b7b4e11b4168353d0123e" +checksum = "0629cbd6b897944899b1f10496d9c4a7ac5878d45fd61bc22e9e79bfbbc29597" dependencies = [ "proc-macro-crate 1.3.0", "proc-macro2", @@ -4120,16 +4127,15 @@ checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" [[package]] name = "opendal" -version = "0.26.2" +version = "0.27.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c40ed33cc9fed187ce8293587416e0afd6ac9fcde17f2a20ad0dca14dd685ebe" +checksum = "ef6f7b936f2f8483e19643357cb50d9ec9a49c506971ef69ca676913cf5afd91" dependencies = [ "anyhow", "async-compat", "async-trait", - "backon 0.2.0", + "backon", "base64 0.21.0", - "bincode 2.0.0-rc.2", "bytes", "flagset", "futures", @@ -4146,7 +4152,7 @@ dependencies = [ "reqwest", "serde", "serde_json", - "time 0.3.18", + "time 0.3.17", "tokio", "ureq", "uuid", @@ -4494,6 +4500,15 @@ dependencies = [ "base64 0.13.1", ] +[[package]] +name = "pem-rfc7468" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24d159833a9105500e0398934e205e0773f0b27529557134ecfc51c27646adac" +dependencies = [ + "base64ct", +] + [[package]] name = "percent-encoding" version = "2.2.0" @@ -4618,6 +4633,28 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" +[[package]] +name = "pkcs1" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eff33bdbdfc54cc98a2eca766ebdec3e1b8fb7387523d5c9c9a2891da856f719" +dependencies = [ + "der", + "pkcs8", + "spki", + "zeroize", +] + +[[package]] +name = "pkcs8" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9eca2c590a5f85da82668fa685c09ce2888b9430e83299debf1f34b65fd4a4ba" +dependencies = [ + "der", + "spki", +] + [[package]] name = "pkg-config" version = "0.3.26" @@ -4729,9 +4766,9 @@ dependencies = [ [[package]] name = "pprof" -version = "0.11.0" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e20150f965e0e4c925982b9356da71c84bcd56cb66ef4e894825837cbcf6613e" +checksum = "196ded5d4be535690899a4631cc9f18cdc41b7ebf24a79400f46f48e49a11059" dependencies = [ "backtrace", "cfg-if", @@ -4739,7 +4776,7 @@ dependencies = [ "inferno", "libc", "log", - "nix 0.24.3", + "nix 0.26.2", "once_cell", "parking_lot 0.12.1", "smallvec", @@ -4847,9 +4884,9 @@ checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" [[package]] name = "proc-macro2" -version = "1.0.49" +version = "1.0.51" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57a8eca9f9c4ffde41714334dee777596264c7825420f521abc92b5b5deb63a5" +checksum = "5d727cae5b39d21da60fa540906919ad737832fe0b1c165da3a34d6548c849d6" dependencies = [ "unicode-ident", ] @@ -4906,7 +4943,7 @@ dependencies = [ "reqwest", "serde", "serde_json", - "time 0.3.18", + "time 0.3.17", "url", ] @@ -5355,12 +5392,12 @@ dependencies = [ [[package]] name = "reqsign" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e0154ced5e44389686689a3c96c49ca5a70ad9c708e0989982adae2e0378bbf" +checksum = "ef4d5fefeaaa1e64f4aabb79da4ea68bf6d0e7935ad927728280d2a8e95735fc" dependencies = [ "anyhow", - "backon 0.4.0", + "backon", "base64 0.21.0", "bytes", "dirs", @@ -5373,12 +5410,14 @@ dependencies = [ "once_cell", "percent-encoding", "quick-xml 0.27.1", + "rand 0.8.5", + "rsa", "rust-ini", "serde", "serde_json", "sha1", "sha2", - "time 0.3.18", + "time 0.3.17", "ureq", ] @@ -5435,9 +5474,9 @@ checksum = "4389f1d5789befaf6029ebd9f7dac4af7f7e3d61b69d4f30e2ac02b57e7712b0" [[package]] name = "rgb" -version = "0.8.35" +version = "0.8.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7495acf66551cdb696b7711408144bcd3194fc78e32f3a09e809bfe7dd4a7ce3" +checksum = "20ec2d3e3fc7a92ced357df9cebd5a10b6fb2aa1ee797bf7e9ce2f17dffc8f59" dependencies = [ "bytemuck", ] @@ -5656,6 +5695,7 @@ dependencies = [ "criterion", "darwin-libproc", "derivative", + "easy-ext", "fixedbitset", "futures", "futures-async-stream", @@ -5827,6 +5867,7 @@ dependencies = [ "aws-sdk-s3", "aws-smithy-http", "aws-types", + "bincode", "byteorder", "bytes", "chrono", @@ -6403,6 +6444,7 @@ dependencies = [ "async-trait", "async_stack_trace", "bytes", + "derivative", "dyn-clone", "either", "enum-as-inner", @@ -6514,6 +6556,27 @@ version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3582f63211428f83597b51b2ddb88e2a91a9d52d12831f9d08f5e624e8977422" +[[package]] +name = "rsa" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89b3896c9b7790b70a9aa314a30e4ae114200992a19c96cbe0ca6070edd32ab8" +dependencies = [ + "byteorder", + "digest", + "num-bigint-dig", + "num-integer", + "num-iter", + "num-traits", + "pkcs1", + "pkcs8", + "rand_core 0.6.4", + "sha2", + "signature", + "subtle", + "zeroize", +] + [[package]] name = "rust-ini" version = "0.18.0" @@ -6809,7 +6872,7 @@ dependencies = [ "serde", "serde_json", "serde_with_macros", - "time 0.3.18", + "time 0.3.17", ] [[package]] @@ -6946,6 +7009,16 @@ dependencies = [ "libc", ] +[[package]] +name = "signature" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fe458c98333f9c8152221191a77e2a44e8325d0193484af2e9421a53019e57d" +dependencies = [ + "digest", + "rand_core 0.6.4", +] + [[package]] name = "simd-json" version = "0.7.0" @@ -6980,7 +7053,7 @@ dependencies = [ "num-bigint", "num-traits", "thiserror", - "time 0.3.18", + "time 0.3.17", ] [[package]] @@ -7012,9 +7085,9 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.7" +version = "0.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4614a76b2a8be0058caa9dbbaf66d988527d86d003c11a94fbd335d7661edcef" +checksum = "6528351c9bc8ab22353f9d776db39a20288e8d6c37ef8cfe3317cf875eecfc2d" dependencies = [ "autocfg", ] @@ -7096,6 +7169,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", +] + [[package]] name = "sqllogictest" version = "0.11.2" @@ -7429,9 +7512,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.18" +version = "0.3.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af0097eaf301d576d0b2aead7a59facab6d53cc636340f0291fab8446a2e8613" +checksum = "a561bf4617eebd33bca6434b988f39ed798e527f51a1e797d0ee4f61c0a38376" dependencies = [ "itoa", "libc", @@ -7612,12 +7695,11 @@ dependencies = [ [[package]] name = "tokio-stream" version = "0.1.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d660770404473ccd7bc9f8b28494a811bc18542b915c0855c51e8f419d5223ce" +source = "git+https://github.com/madsim-rs/tokio.git?rev=0c25710#0c25710a9d4e415a47820a4bdff11bd30e377572" dependencies = [ "futures-core", + "madsim-tokio", "pin-project-lite", - "tokio", ] [[package]] @@ -7883,7 +7965,7 @@ dependencies = [ "sharded-slab", "smallvec", "thread_local", - "time 0.3.18", + "time 0.3.17", "tracing", "tracing-core", "tracing-log", @@ -8116,12 +8198,6 @@ version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" -[[package]] -name = "virtue" -version = "0.0.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b60dcd6a64dd45abf9bd426970c9843726da7fc08f44cd6fcebf68c21220a63" - [[package]] name = "waker-fn" version = "1.1.0" @@ -8578,7 +8654,7 @@ dependencies = [ "stable_deref_trait", "strum", "syn", - "time 0.3.18", + "time 0.3.17", "tokio", "tokio-stream", "tokio-util", diff --git a/Cargo.toml b/Cargo.toml index 7fb3e579492f1..e7fc277f3fa09 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -119,6 +119,14 @@ incremental = false [patch.crates-io] quanta = { git = "https://github.com/madsim-rs/quanta.git", rev = "a819877" } getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "cc95ee3" } +tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "0c25710" } 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 = "87ca1dc" } postgres-types = { git = "https://github.com/madsim-rs/rust-postgres.git", rev = "87ca1dc" } + +# TODO: remove these patches when arrow releases v34 +# we need this commit for handling batch with no columns +# https://github.com/apache/arrow-rs/commit/ea48b9571f88bfbced60f9790ae2a7102502870e +arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "9a6c516" } +arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "9a6c516" } +arrow-flight = { git = "https://github.com/apache/arrow-rs.git", rev = "9a6c516" } diff --git a/ci/Dockerfile b/ci/Dockerfile index da9ba9dc5fae3..cf19293e861ca 100644 --- a/ci/Dockerfile +++ b/ci/Dockerfile @@ -5,7 +5,7 @@ ENV LANG en_US.utf8 ARG RUST_TOOLCHAIN RUN apt-get update -yy && \ - DEBIAN_FRONTEND=noninteractive apt-get -y install make build-essential cmake protobuf-compiler curl parallel \ + DEBIAN_FRONTEND=noninteractive apt-get -y install make build-essential cmake protobuf-compiler curl parallel python3 python3-pip \ openssl libssl-dev libsasl2-dev libcurl4-openssl-dev pkg-config bash openjdk-11-jdk wget unzip git tmux lld postgresql-client kafkacat netcat mysql-client \ maven -yy \ && rm -rf /var/lib/{apt,dpkg,cache,log}/ @@ -23,6 +23,9 @@ WORKDIR /risingwave ENV PATH /root/.cargo/bin/:$PATH +# install python dependencies +RUN pip3 install pyarrow + # add required rustup components RUN rustup component add rustfmt llvm-tools-preview clippy diff --git a/ci/build-ci-image.sh b/ci/build-ci-image.sh index 875c20a8d7647..ffb8b0fbe8d0b 100755 --- a/ci/build-ci-image.sh +++ b/ci/build-ci-image.sh @@ -14,7 +14,7 @@ export RUST_TOOLCHAIN=$(cat ../rust-toolchain) # !!! CHANGE THIS WHEN YOU WANT TO BUMP CI IMAGE !!! # # AND ALSO docker-compose.yml # ###################################################### -export BUILD_ENV_VERSION=v20230220 +export BUILD_ENV_VERSION=v20230221_01 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 d33b7163f372f..6b4493a0fd85c 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -24,13 +24,17 @@ services: - MYSQL_USER=mysqluser - MYSQL_PASSWORD=mysqlpw healthcheck: - test: [ "CMD-SHELL", "mysqladmin ping -h 127.0.0.1 -u root -p123456" ] + test: + [ + "CMD-SHELL", + "mysqladmin ping -h 127.0.0.1 -u root -p123456" + ] interval: 5s timeout: 5s retries: 5 source-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230220 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230221_01 depends_on: - mysql - db @@ -38,7 +42,7 @@ services: - ..:/risingwave sink-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230220 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230221_01 depends_on: - mysql - db @@ -46,12 +50,12 @@ services: - ..:/risingwave rw-build-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230220 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230221_01 volumes: - ..:/risingwave regress-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230220 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230221_01 depends_on: db: condition: service_healthy diff --git a/ci/scripts/build.sh b/ci/scripts/build.sh index b04ba2aad59c6..a8579c5c7af2c 100755 --- a/ci/scripts/build.sh +++ b/ci/scripts/build.sh @@ -42,9 +42,11 @@ cargo build \ -p risingwave_sqlsmith \ -p risingwave_compaction_test \ -p risingwave_backup_cmd \ + -p risingwave_java_binding \ --features "static-link static-log-level" --profile "$profile" -artifacts=(risingwave sqlsmith compaction-test backup-restore risingwave_regress_test risedev-dev delete-range-test) +# the file name suffix of artifact for risingwave_java_binding is so only for linux. It is dylib for MacOS +artifacts=(risingwave sqlsmith compaction-test backup-restore risingwave_regress_test risedev-dev delete-range-test librisingwave_java_binding.so) echo "--- Compress debug info for artifacts" echo -n "${artifacts[*]}" | parallel -d ' ' "objcopy --compress-debug-sections=zlib-gnu target/$target/{} && echo \"compressed {}\"" diff --git a/ci/scripts/e2e-iceberg-sink-test.sh b/ci/scripts/e2e-iceberg-sink-test.sh index 67a34a0e1466c..4fe338dbffe1e 100755 --- a/ci/scripts/e2e-iceberg-sink-test.sh +++ b/ci/scripts/e2e-iceberg-sink-test.sh @@ -52,7 +52,7 @@ sleep 1 # prepare minio iceberg sink echo "--- preparing iceberg" .risingwave/bin/mcli -C .risingwave/config/mcli mb hummock-minio/iceberg -wget https://archive.apache.org/dist/spark/spark-3.3.1/spark-3.3.1-bin-hadoop3.tgz +wget https://iceberg-ci-spark-dist.s3.amazonaws.com/spark-3.3.1-bin-hadoop3.tgz tar -xf spark-3.3.1-bin-hadoop3.tgz --no-same-owner DEPENDENCIES=org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.0.0,org.apache.hadoop:hadoop-aws:3.3.2 spark-3.3.1-bin-hadoop3/bin/spark-sql --packages $DEPENDENCIES \ diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index ad5854a2fd8bd..9eb8d72107b1f 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -87,6 +87,7 @@ echo "--- starting risingwave cluster with connector node" cargo make ci-start ci-1cn-1fe echo "--- testing sinks" +sqllogictest -p 4566 -d dev './e2e_test/sink/append_only_sink.slt' sqllogictest -p 4566 -d dev './e2e_test/sink/create_sink_as.slt' sqllogictest -p 4566 -d dev './e2e_test/sink/blackhole_sink.slt' sleep 1 diff --git a/ci/scripts/java-binding-test.sh b/ci/scripts/java-binding-test.sh new file mode 100755 index 0000000000000..9dca6068a6915 --- /dev/null +++ b/ci/scripts/java-binding-test.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash + +# Exits as soon as any line fails. +set -euo pipefail + +source ci/scripts/common.env.sh + +while getopts 'p:' opt; do + case ${opt} in + p ) + profile=$OPTARG + ;; + \? ) + echo "Invalid Option: -$OPTARG" 1>&2 + exit 1 + ;; + : ) + echo "Invalid option: $OPTARG requires an argument" 1>&2 + ;; + esac +done +shift $((OPTIND -1)) + +echo "--- Download artifacts" +mkdir -p target/debug +buildkite-agent artifact download risingwave-"$profile" target/debug/ +buildkite-agent artifact download risedev-dev-"$profile" target/debug/ +buildkite-agent artifact download librisingwave_java_binding.so-"$profile" target/debug +mv target/debug/risingwave-"$profile" target/debug/risingwave +mv target/debug/risedev-dev-"$profile" target/debug/risedev-dev +mv target/debug/librisingwave_java_binding.so-"$profile" target/debug/librisingwave_java_binding.so + +echo "--- Adjust permission" +chmod +x ./target/debug/risingwave +chmod +x ./target/debug/risedev-dev +chmod +x ./target/debug/librisingwave_java_binding.so + +echo "--- Generate RiseDev CI config" +cp ci/risedev-components.ci.source.env risedev-components.user.env + +echo "--- Prepare RiseDev dev cluster" +cargo make pre-start-dev +cargo make link-all-in-one-binaries + +echo "--- starting risingwave cluster" +cargo make ci-start java-binding-demo + +echo "--- Build java binding demo" +cargo make build-java-binding-java + +echo "--- ingest data and run java binding" +cargo make ingest-data-and-run-java-binding + +echo "--- Kill cluster" +cargo make ci-kill diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index da50b2656604c..93dc93447de7d 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -55,6 +55,12 @@ sqllogictest -p 4566 -d dev './e2e_test/batch/**/*.slt' --junit "batch-${profile sqllogictest -p 4566 -d dev './e2e_test/database/prepare.slt' sqllogictest -p 4566 -d test './e2e_test/database/test.slt' +echo "--- e2e, ci-3cn-1fe, udf" +python3 e2e_test/udf/test.py & +sleep 2 +sqllogictest -p 4566 -d dev './e2e_test/udf/python.slt' +pkill python3 + echo "--- Kill cluster" cargo make ci-kill diff --git a/ci/workflows/main.yml b/ci/workflows/main.yml index f2eae9d16375a..d9eca30a97786 100644 --- a/ci/workflows/main.yml +++ b/ci/workflows/main.yml @@ -301,6 +301,19 @@ steps: timeout_in_minutes: 5 retry: *auto-retry + - label: "e2e java-binding test (at release)" + command: "ci/scripts/java-binding-test.sh -p ci-release" + depends_on: "build-release" + plugins: + - gencer/cache#v2.4.10: *cargo-cache + - docker-compose#v4.9.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 5 + retry: *auto-retry + - label: "release" command: "ci/scripts/release.sh" if: build.tag != null diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index e6b36f3f68e78..8ce43044fb3e5 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -171,6 +171,19 @@ steps: timeout_in_minutes: 5 retry: *auto-retry + - label: "e2e java-binding test" + command: "ci/scripts/java-binding-test.sh -p ci-dev" + depends_on: "build" + plugins: + - gencer/cache#v2.4.10: *cargo-cache + - docker-compose#v4.9.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 5 + retry: *auto-retry + - label: "regress test" command: "ci/scripts/regress-test.sh -p ci-dev" depends_on: "build" diff --git a/dashboard/proto/gen/catalog.ts b/dashboard/proto/gen/catalog.ts index f032be4bd55b7..09811bac319ee 100644 --- a/dashboard/proto/gen/catalog.ts +++ b/dashboard/proto/gen/catalog.ts @@ -82,6 +82,7 @@ export interface StreamSourceInfo { protoMessageName: string; csvDelimiter: number; csvHasHeader: boolean; + upsertAvroPrimaryKey: string; } export interface Source { @@ -379,6 +380,7 @@ function createBaseStreamSourceInfo(): StreamSourceInfo { protoMessageName: "", csvDelimiter: 0, csvHasHeader: false, + upsertAvroPrimaryKey: "", }; } @@ -391,6 +393,7 @@ export const StreamSourceInfo = { protoMessageName: isSet(object.protoMessageName) ? String(object.protoMessageName) : "", csvDelimiter: isSet(object.csvDelimiter) ? Number(object.csvDelimiter) : 0, csvHasHeader: isSet(object.csvHasHeader) ? Boolean(object.csvHasHeader) : false, + upsertAvroPrimaryKey: isSet(object.upsertAvroPrimaryKey) ? String(object.upsertAvroPrimaryKey) : "", }; }, @@ -402,6 +405,7 @@ export const StreamSourceInfo = { message.protoMessageName !== undefined && (obj.protoMessageName = message.protoMessageName); message.csvDelimiter !== undefined && (obj.csvDelimiter = Math.round(message.csvDelimiter)); message.csvHasHeader !== undefined && (obj.csvHasHeader = message.csvHasHeader); + message.upsertAvroPrimaryKey !== undefined && (obj.upsertAvroPrimaryKey = message.upsertAvroPrimaryKey); return obj; }, @@ -413,6 +417,7 @@ export const StreamSourceInfo = { message.protoMessageName = object.protoMessageName ?? ""; message.csvDelimiter = object.csvDelimiter ?? 0; message.csvHasHeader = object.csvHasHeader ?? false; + message.upsertAvroPrimaryKey = object.upsertAvroPrimaryKey ?? ""; return message; }, }; diff --git a/dashboard/proto/gen/common.ts b/dashboard/proto/gen/common.ts index 0e5c3f88ce789..b87219cf9e71e 100644 --- a/dashboard/proto/gen/common.ts +++ b/dashboard/proto/gen/common.ts @@ -8,6 +8,7 @@ export const WorkerType = { COMPUTE_NODE: "COMPUTE_NODE", RISE_CTL: "RISE_CTL", COMPACTOR: "COMPACTOR", + META: "META", UNRECOGNIZED: "UNRECOGNIZED", } as const; @@ -30,6 +31,9 @@ export function workerTypeFromJSON(object: any): WorkerType { case 4: case "COMPACTOR": return WorkerType.COMPACTOR; + case 5: + case "META": + return WorkerType.META; case -1: case "UNRECOGNIZED": default: @@ -49,6 +53,8 @@ export function workerTypeToJSON(object: WorkerType): string { return "RISE_CTL"; case WorkerType.COMPACTOR: return "COMPACTOR"; + case WorkerType.META: + return "META"; case WorkerType.UNRECOGNIZED: default: return "UNRECOGNIZED"; diff --git a/dashboard/proto/gen/expr.ts b/dashboard/proto/gen/expr.ts index d5e49cb52885f..66bc96aea0d75 100644 --- a/dashboard/proto/gen/expr.ts +++ b/dashboard/proto/gen/expr.ts @@ -130,8 +130,14 @@ export const ExprNode_Type = { ARRAY_APPEND: "ARRAY_APPEND", ARRAY_PREPEND: "ARRAY_PREPEND", FORMAT_TYPE: "FORMAT_TYPE", + /** JSONB_ACCESS_INNER - jsonb -> int, jsonb -> text, jsonb #> text[] that returns jsonb */ + JSONB_ACCESS_INNER: "JSONB_ACCESS_INNER", + /** JSONB_ACCESS_STR - jsonb ->> int, jsonb ->> text, jsonb #>> text[] that returns text */ + JSONB_ACCESS_STR: "JSONB_ACCESS_STR", + JSONB_TYPEOF: "JSONB_TYPEOF", + JSONB_ARRAY_LENGTH: "JSONB_ARRAY_LENGTH", /** - * VNODE - Non-pure functions below (> 600) + * VNODE - Non-pure functions below (> 1000) * ------------------------ * Internal functions */ @@ -402,6 +408,18 @@ export function exprNode_TypeFromJSON(object: any): ExprNode_Type { case 534: case "FORMAT_TYPE": return ExprNode_Type.FORMAT_TYPE; + case 600: + case "JSONB_ACCESS_INNER": + return ExprNode_Type.JSONB_ACCESS_INNER; + case 601: + case "JSONB_ACCESS_STR": + return ExprNode_Type.JSONB_ACCESS_STR; + case 602: + case "JSONB_TYPEOF": + return ExprNode_Type.JSONB_TYPEOF; + case 603: + case "JSONB_ARRAY_LENGTH": + return ExprNode_Type.JSONB_ARRAY_LENGTH; case 1101: case "VNODE": return ExprNode_Type.VNODE; @@ -590,6 +608,14 @@ export function exprNode_TypeToJSON(object: ExprNode_Type): string { return "ARRAY_PREPEND"; case ExprNode_Type.FORMAT_TYPE: return "FORMAT_TYPE"; + case ExprNode_Type.JSONB_ACCESS_INNER: + return "JSONB_ACCESS_INNER"; + case ExprNode_Type.JSONB_ACCESS_STR: + return "JSONB_ACCESS_STR"; + case ExprNode_Type.JSONB_TYPEOF: + return "JSONB_TYPEOF"; + case ExprNode_Type.JSONB_ARRAY_LENGTH: + return "JSONB_ARRAY_LENGTH"; case ExprNode_Type.VNODE: return "VNODE"; case ExprNode_Type.NOW: diff --git a/dashboard/proto/gen/hummock.ts b/dashboard/proto/gen/hummock.ts index 2deaff7c6e9a6..d63dc2e133f7d 100644 --- a/dashboard/proto/gen/hummock.ts +++ b/dashboard/proto/gen/hummock.ts @@ -56,6 +56,8 @@ export interface SstableInfo { totalKeyCount: number; /** When a SST is divided, its divide_version will increase one. */ divideVersion: number; + minEpoch: number; + maxEpoch: number; } export interface OverlappingLevel { @@ -840,6 +842,8 @@ function createBaseSstableInfo(): SstableInfo { staleKeyCount: 0, totalKeyCount: 0, divideVersion: 0, + minEpoch: 0, + maxEpoch: 0, }; } @@ -854,6 +858,8 @@ export const SstableInfo = { staleKeyCount: isSet(object.staleKeyCount) ? Number(object.staleKeyCount) : 0, totalKeyCount: isSet(object.totalKeyCount) ? Number(object.totalKeyCount) : 0, divideVersion: isSet(object.divideVersion) ? Number(object.divideVersion) : 0, + minEpoch: isSet(object.minEpoch) ? Number(object.minEpoch) : 0, + maxEpoch: isSet(object.maxEpoch) ? Number(object.maxEpoch) : 0, }; }, @@ -871,6 +877,8 @@ export const SstableInfo = { message.staleKeyCount !== undefined && (obj.staleKeyCount = Math.round(message.staleKeyCount)); message.totalKeyCount !== undefined && (obj.totalKeyCount = Math.round(message.totalKeyCount)); message.divideVersion !== undefined && (obj.divideVersion = Math.round(message.divideVersion)); + message.minEpoch !== undefined && (obj.minEpoch = Math.round(message.minEpoch)); + message.maxEpoch !== undefined && (obj.maxEpoch = Math.round(message.maxEpoch)); return obj; }, @@ -886,6 +894,8 @@ export const SstableInfo = { message.staleKeyCount = object.staleKeyCount ?? 0; message.totalKeyCount = object.totalKeyCount ?? 0; message.divideVersion = object.divideVersion ?? 0; + message.minEpoch = object.minEpoch ?? 0; + message.maxEpoch = object.maxEpoch ?? 0; return message; }, }; diff --git a/dashboard/proto/gen/plan_common.ts b/dashboard/proto/gen/plan_common.ts index bb5e8b7568f69..01cf2a88bfde3 100644 --- a/dashboard/proto/gen/plan_common.ts +++ b/dashboard/proto/gen/plan_common.ts @@ -136,6 +136,8 @@ export const RowFormatType = { CSV: "CSV", NATIVE: "NATIVE", DEBEZIUM_AVRO: "DEBEZIUM_AVRO", + UPSERT_JSON: "UPSERT_JSON", + UPSERT_AVRO: "UPSERT_AVRO", UNRECOGNIZED: "UNRECOGNIZED", } as const; @@ -173,6 +175,12 @@ export function rowFormatTypeFromJSON(object: any): RowFormatType { case 9: case "DEBEZIUM_AVRO": return RowFormatType.DEBEZIUM_AVRO; + case 10: + case "UPSERT_JSON": + return RowFormatType.UPSERT_JSON; + case 11: + case "UPSERT_AVRO": + return RowFormatType.UPSERT_AVRO; case -1: case "UNRECOGNIZED": default: @@ -202,6 +210,10 @@ export function rowFormatTypeToJSON(object: RowFormatType): string { return "NATIVE"; case RowFormatType.DEBEZIUM_AVRO: return "DEBEZIUM_AVRO"; + case RowFormatType.UPSERT_JSON: + return "UPSERT_JSON"; + case RowFormatType.UPSERT_AVRO: + return "UPSERT_AVRO"; case RowFormatType.UNRECOGNIZED: default: return "UNRECOGNIZED"; diff --git a/dashboard/proto/gen/stream_plan.ts b/dashboard/proto/gen/stream_plan.ts index c950828c55cef..697a3c5a870fb 100644 --- a/dashboard/proto/gen/stream_plan.ts +++ b/dashboard/proto/gen/stream_plan.ts @@ -827,9 +827,14 @@ export interface StreamNode { fields: Field[]; } +/** + * The property of an edge in the fragment graph. + * This is essientially a "logical" version of `Dispatcher`. See the doc of `Dispatcher` for more details. + */ export interface DispatchStrategy { type: DispatcherType; - columnIndices: number[]; + distKeyIndices: number[]; + outputIndices: number[]; } /** @@ -842,7 +847,13 @@ export interface Dispatcher { * Indices of the columns to be used for hashing. * For dispatcher types other than HASH, this is ignored. */ - columnIndices: number[]; + distKeyIndices: number[]; + /** + * Indices of the columns to output. + * In most cases, this contains all columns in the input. But for some cases like MV on MV or + * schema change, we may only output a subset of the columns. + */ + outputIndices: number[]; /** * The hash mapping for consistent hash. * For dispatcher types other than HASH, this is ignored. @@ -3870,24 +3881,30 @@ export const StreamNode = { }; function createBaseDispatchStrategy(): DispatchStrategy { - return { type: DispatcherType.UNSPECIFIED, columnIndices: [] }; + return { type: DispatcherType.UNSPECIFIED, distKeyIndices: [], outputIndices: [] }; } export const DispatchStrategy = { fromJSON(object: any): DispatchStrategy { return { type: isSet(object.type) ? dispatcherTypeFromJSON(object.type) : DispatcherType.UNSPECIFIED, - columnIndices: Array.isArray(object?.columnIndices) ? object.columnIndices.map((e: any) => Number(e)) : [], + distKeyIndices: Array.isArray(object?.distKeyIndices) ? object.distKeyIndices.map((e: any) => Number(e)) : [], + outputIndices: Array.isArray(object?.outputIndices) ? object.outputIndices.map((e: any) => Number(e)) : [], }; }, toJSON(message: DispatchStrategy): unknown { const obj: any = {}; message.type !== undefined && (obj.type = dispatcherTypeToJSON(message.type)); - if (message.columnIndices) { - obj.columnIndices = message.columnIndices.map((e) => Math.round(e)); + if (message.distKeyIndices) { + obj.distKeyIndices = message.distKeyIndices.map((e) => Math.round(e)); } else { - obj.columnIndices = []; + obj.distKeyIndices = []; + } + if (message.outputIndices) { + obj.outputIndices = message.outputIndices.map((e) => Math.round(e)); + } else { + obj.outputIndices = []; } return obj; }, @@ -3895,7 +3912,8 @@ export const DispatchStrategy = { fromPartial, I>>(object: I): DispatchStrategy { const message = createBaseDispatchStrategy(); message.type = object.type ?? DispatcherType.UNSPECIFIED; - message.columnIndices = object.columnIndices?.map((e) => e) || []; + message.distKeyIndices = object.distKeyIndices?.map((e) => e) || []; + message.outputIndices = object.outputIndices?.map((e) => e) || []; return message; }, }; @@ -3903,7 +3921,8 @@ export const DispatchStrategy = { function createBaseDispatcher(): Dispatcher { return { type: DispatcherType.UNSPECIFIED, - columnIndices: [], + distKeyIndices: [], + outputIndices: [], hashMapping: undefined, dispatcherId: 0, downstreamActorId: [], @@ -3914,7 +3933,8 @@ export const Dispatcher = { fromJSON(object: any): Dispatcher { return { type: isSet(object.type) ? dispatcherTypeFromJSON(object.type) : DispatcherType.UNSPECIFIED, - columnIndices: Array.isArray(object?.columnIndices) ? object.columnIndices.map((e: any) => Number(e)) : [], + distKeyIndices: Array.isArray(object?.distKeyIndices) ? object.distKeyIndices.map((e: any) => Number(e)) : [], + outputIndices: Array.isArray(object?.outputIndices) ? object.outputIndices.map((e: any) => Number(e)) : [], hashMapping: isSet(object.hashMapping) ? ActorMapping.fromJSON(object.hashMapping) : undefined, dispatcherId: isSet(object.dispatcherId) ? Number(object.dispatcherId) : 0, downstreamActorId: Array.isArray(object?.downstreamActorId) @@ -3926,10 +3946,15 @@ export const Dispatcher = { toJSON(message: Dispatcher): unknown { const obj: any = {}; message.type !== undefined && (obj.type = dispatcherTypeToJSON(message.type)); - if (message.columnIndices) { - obj.columnIndices = message.columnIndices.map((e) => Math.round(e)); + if (message.distKeyIndices) { + obj.distKeyIndices = message.distKeyIndices.map((e) => Math.round(e)); } else { - obj.columnIndices = []; + obj.distKeyIndices = []; + } + if (message.outputIndices) { + obj.outputIndices = message.outputIndices.map((e) => Math.round(e)); + } else { + obj.outputIndices = []; } message.hashMapping !== undefined && (obj.hashMapping = message.hashMapping ? ActorMapping.toJSON(message.hashMapping) : undefined); @@ -3945,7 +3970,8 @@ export const Dispatcher = { fromPartial, I>>(object: I): Dispatcher { const message = createBaseDispatcher(); message.type = object.type ?? DispatcherType.UNSPECIFIED; - message.columnIndices = object.columnIndices?.map((e) => e) || []; + message.distKeyIndices = object.distKeyIndices?.map((e) => e) || []; + message.outputIndices = object.outputIndices?.map((e) => e) || []; message.hashMapping = (object.hashMapping !== undefined && object.hashMapping !== null) ? ActorMapping.fromPartial(object.hashMapping) : undefined; diff --git a/e2e_test/batch/explain.slt b/e2e_test/batch/explain.slt index 9a42b8e3af986..388f92440ebae 100644 --- a/e2e_test/batch/explain.slt +++ b/e2e_test/batch/explain.slt @@ -1,11 +1,11 @@ statement ok -create table t(v int); +create table t (v int) with ( appendonly = 'true' ); statement ok explain create index i on t(v); statement ok -explain create sink sink_t from t with ( connector = 'kafka' ) +explain create sink sink_t from t with ( connector = 'kafka', format = 'append_only' ) statement ok drop table t; diff --git a/e2e_test/batch/functions/now.slt.part b/e2e_test/batch/functions/now.slt.part index 8d3ce7867b3d9..7c2eb82f45921 100644 --- a/e2e_test/batch/functions/now.slt.part +++ b/e2e_test/batch/functions/now.slt.part @@ -11,11 +11,10 @@ statement ok insert into t values(now()); # constant eval of now in batch plan -# query T -# explain select now() + interval '1 hour' = now() + interval '30 minutes' + interval '30 minutes' true; -# ---- -# BatchProject { exprs: [true:Boolean] } -# └─BatchValues { rows: [[]] } +query T +explain select now() + interval '1 hour' = now() + interval '30 minutes' + interval '30 minutes' true; +---- +BatchValues { rows: [[true:Boolean]] } statement ok drop table tz diff --git a/e2e_test/batch/types/array_ty.slt.part b/e2e_test/batch/types/array_ty.slt.part index 3839d261dcfdd..5f1a58e685319 100644 --- a/e2e_test/batch/types/array_ty.slt.part +++ b/e2e_test/batch/types/array_ty.slt.part @@ -84,6 +84,20 @@ select max(ARRAY[1, v1*2]) from t; ---- {1,6} +query T +select CAST(NULL as bool[]) from t; +---- +NULL +NULL +NULL + +query T +select array[false, false] from t; +---- +{f,f} +{f,f} +{f,f} + statement ok drop table t; diff --git a/e2e_test/batch/types/jsonb.slt.part b/e2e_test/batch/types/jsonb.slt.part index f0d890bc9d877..54d29abaae6b7 100644 --- a/e2e_test/batch/types/jsonb.slt.part +++ b/e2e_test/batch/types/jsonb.slt.part @@ -83,5 +83,41 @@ select null::jsonb::bool; ---- NULL +query T +select jsonb_array_element(jsonb_object_field('{"k2":[2,true,4]}', 'k2'), -2)::bool; +---- +t + +# Note the difference between access text directly vs access jsonb then cast to text. +query TTT +with t(v1) as (values (null::jsonb), ('null'), ('true'), ('1'), ('"a"'), ('[]'), ('{}')), + j(v1) as (select ('{"k":' || v1::varchar || '}')::jsonb from t) +select + jsonb_object_field_text(v1, 'k'), + jsonb_object_field(v1, 'k')::varchar, + jsonb_typeof(jsonb_object_field(v1, 'k')) +from j order by 2; +---- +a "a" string +1 1 number +[] [] array +NULL null null +true true boolean +{} {} object +NULL NULL NULL + +query T +select jsonb_array_element_text('true'::jsonb, 2); +---- +NULL + +query I +select jsonb_array_length('[7, 2]'); +---- +2 + +statement error cannot get array length +select jsonb_array_length('null'); + statement ok drop table t; diff --git a/e2e_test/ddl/function.slt b/e2e_test/ddl/function.slt deleted file mode 100644 index 5f7dea9487f72..0000000000000 --- a/e2e_test/ddl/function.slt +++ /dev/null @@ -1,27 +0,0 @@ -# TODO: check the service on creation - -# Create a function. -statement ok -create function func(int, int) returns int as 'http://localhost:8815' language arrow_flight; - -# Create a function with the same name but different arguments. -statement ok -create function func(int) returns int as 'http://localhost:8815' language arrow_flight; - -# Create a function with the same name and arguments. -statement error exists -create function func(int) returns int as 'http://localhost:8815' language arrow_flight; - -# TODO: drop function without arguments - -# # Drop a function but ambiguous. -# statement error is not unique -# drop function func; - -# Drop a function -statement ok -drop function func(int); - -# Drop a function -statement ok -drop function func(int, int); diff --git a/e2e_test/ddl/invalid_operation.slt b/e2e_test/ddl/invalid_operation.slt index 97696042accd7..9d5ef49c7272b 100644 --- a/e2e_test/ddl/invalid_operation.slt +++ b/e2e_test/ddl/invalid_operation.slt @@ -62,7 +62,7 @@ drop view not_exists.not_exists.not_exists; # 4.1. table statement ok -create table t (v int); +create table t (v int primary key); statement error Use `DROP TABLE` drop materialized view t; diff --git a/e2e_test/ddl/table.slt b/e2e_test/ddl/table.slt index 6cc3d7325c424..e941c534e8ba8 100644 --- a/e2e_test/ddl/table.slt +++ b/e2e_test/ddl/table.slt @@ -35,10 +35,10 @@ statement ok explain select v2 from ddl_t; statement ok -explain create sink sink_t from ddl_t with ( connector = 'kafka' ); +explain create sink sink_t from ddl_t with ( connector = 'kafka', format = 'append_only', force_append_only = 'true' ); statement ok -explain create sink sink_as as select sum(v2) as sum from ddl_t with ( connector = 'kafka' ); +explain create sink sink_as as select sum(v2) as sum from ddl_t with ( connector = 'kafka', format = 'append_only', force_append_only = 'true' ); # Create a mview with duplicated name. statement error diff --git a/e2e_test/generated/README.md b/e2e_test/generated/README.md new file mode 100644 index 0000000000000..2c7b7aa7b3d39 --- /dev/null +++ b/e2e_test/generated/README.md @@ -0,0 +1,11 @@ +# Generated e2e tests + +## docslt + +Generated by + +```bash +cargo run --bin risedev-docslt +``` + +Refer to `src/risedevtool/src/bin/risedev-docslt.rs` for more details. diff --git a/e2e_test/sink/append_only_sink.slt b/e2e_test/sink/append_only_sink.slt new file mode 100644 index 0000000000000..cf4e185d1b28f --- /dev/null +++ b/e2e_test/sink/append_only_sink.slt @@ -0,0 +1,53 @@ +statement ok +create table t1 (v1 int, v2 int); + +statement error No primary key for the upsert sink +create sink s1 from t1 with (connector = 'console'); + +statement ok +create sink s1 as select v1, v2, _row_id from t1 with (connector = 'console'); + +statement ok +create table t2 (v1 int, v2 int primary key); + +statement ok +create sink s2 from t2 with (connector = 'console'); + +statement error No primary key for the upsert sink +create sink s3 as select avg(v1) from t2 with (connector = 'console'); + +statement ok +create sink s3 as select avg(v1) from t2 with (connector = 'console', format = 'append_only', force_append_only = 'true'); + +statement ok +create sink s4 as select avg(v1), v2 from t2 group by v2 with (connector = 'console'); + +statement error The sink cannot be append-only +create sink s5 from t2 with (connector = 'console', format = 'append_only'); + +statement ok +create sink s5 from t2 with (connector = 'console', format = 'append_only', force_append_only = 'true'); + +statement error Cannot force the sink to be append-only +create sink s6 from t2 with (connector = 'console', format = 'upsert', force_append_only = 'true'); + +statement ok +drop sink s1 + +statement ok +drop sink s2 + +statement ok +drop sink s3 + +statement ok +drop sink s4 + +statement ok +drop sink s5 + +statement ok +drop table t1 + +statement ok +drop table t2 diff --git a/e2e_test/sink/blackhole_sink.slt b/e2e_test/sink/blackhole_sink.slt index e3e9810f81f73..01b744a37b8c7 100644 --- a/e2e_test/sink/blackhole_sink.slt +++ b/e2e_test/sink/blackhole_sink.slt @@ -1,5 +1,5 @@ statement ok -CREATE TABLE t5 (v1 int, v2 int); +CREATE TABLE t5 (v1 int primary key, v2 int); statement ok CREATE MATERIALIZED VIEW mv5 AS SELECT * FROM t5; diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index a7c8b6c73205e..8c9d56dc64bf3 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -132,7 +132,8 @@ create table s9 with ( topic = 'avro_bin', properties.bootstrap.server = '127.0.0.1:29092', scan.startup.mode = 'earliest' -) row format avro message 'test_student' row schema location 'file:///risingwave/avro-simple-schema.avsc' +) row format avro +row schema location 'file:///risingwave/avro-simple-schema.avsc' statement ok create table s10 with ( @@ -140,7 +141,8 @@ create table s10 with ( topic = 'avro_c_bin', properties.bootstrap.server = '127.0.0.1:29092', scan.startup.mode = 'earliest' -) row format avro message 'user' row schema location 'file:///risingwave/avro-complex-schema.avsc' +) row format avro +row schema location 'file:///risingwave/avro-complex-schema.avsc' statement ok create table s11 with ( @@ -240,7 +242,8 @@ create source s18 with ( topic = 'avro_c_bin', properties.bootstrap.server = '127.0.0.1:29092', scan.startup.mode = 'earliest' -) row format avro message 'user' row schema location 'file:///risingwave/avro-complex-schema.avsc' +) row format avro +row schema location 'file:///risingwave/avro-complex-schema.avsc' statement ok flush; diff --git a/e2e_test/streaming/join.slt b/e2e_test/streaming/join.slt index 3536b30891f78..8f50fac0eb729 100644 --- a/e2e_test/streaming/join.slt +++ b/e2e_test/streaming/join.slt @@ -167,3 +167,40 @@ drop table t9 statement ok drop table t10 + +statement ok +flush; + +# Regression test for: https://github.com/risingwavelabs/risingwave/issues/8114 +statement ok +create table t1 (uid int, item_id int, event string, name string); + +statement ok +create table t2 (uid int, name string); + +statement ok +create materialized view v as SELECT event, t1.name FROM t1 INNER JOIN t2 WHERE t1.name=t2.name AND t1.event=concat('event_', array_join(array[t2.uid, t1.item_id], '_')); + +statement ok +insert into t1 values (0, 0, 'event_0_0', 'a'), (1, NULL, 'event_1', 'b'), (2, 3, 'event_2_1', 'c'); + +statement ok +insert into t2 values (0, 'a'), (1, 'b'), (2, 'c'); + +statement ok +flush; + +query I rowsort +select * from v; +---- +event_0_0 a +event_1 b + +statement ok +drop materialized view v; + +statement ok +drop table t1 + +statement ok +drop table t2 \ No newline at end of file diff --git a/e2e_test/udf/python.slt b/e2e_test/udf/python.slt new file mode 100644 index 0000000000000..405ac555cb871 --- /dev/null +++ b/e2e_test/udf/python.slt @@ -0,0 +1,49 @@ +# Before running this test: +# python3 e2e_test/udf/test.py + +# TODO: check the service on creation +# Currently whether the function exists in backend and whether the signature matches is checked on execution. Create function will always succeed. + +# Create a function. +statement ok +create function int_42() returns int as 'http://localhost:8815' language arrow_flight; + +statement ok +create function gcd(int, int) returns int as 'http://localhost:8815' language arrow_flight; + +# Create a function with the same name but different arguments. +statement ok +create function gcd(int, int, int) returns int as 'http://localhost:8815' language arrow_flight; + +# Create a function with the same name and arguments. +statement error exists +create function gcd(int, int) returns int as 'http://localhost:8815' language arrow_flight; + +query I +select int_42(); +---- +42 + +query I +select gcd(25, 15); +---- +5 + +query I +select gcd(25, 15, 3); +---- +1 + +# TODO: drop function without arguments + +# # Drop a function but ambiguous. +# statement error is not unique +# drop function gcd; + +# Drop a function +statement ok +drop function gcd(int, int); + +# Drop a function +statement ok +drop function gcd(int, int, int); diff --git a/e2e_test/udf/test.py b/e2e_test/udf/test.py new file mode 100644 index 0000000000000..ddb907629c12f --- /dev/null +++ b/e2e_test/udf/test.py @@ -0,0 +1,29 @@ +import sys +sys.path.append('src/udf/python') # noqa + +from risingwave.udf import udf, UdfServer + + +@udf(input_types=[], result_type='INT') +def int_42() -> int: + return 42 + + +@udf(input_types=['INT', 'INT'], result_type='INT') +def gcd(x: int, y: int) -> int: + while y != 0: + (x, y) = (y, x % y) + return x + + +@udf(name='gcd', input_types=['INT', 'INT', 'INT'], result_type='INT') +def gcd3(x: int, y: int, z: int) -> int: + return gcd(gcd(x, y), z) + + +if __name__ == '__main__': + server = UdfServer() + server.add_function(int_42) + server.add_function(gcd) + server.add_function(gcd3) + server.serve() diff --git a/grafana/risingwave-dashboard.dashboard.py b/grafana/risingwave-dashboard.dashboard.py index 64a5b50dedeee..6361503f52345 100644 --- a/grafana/risingwave-dashboard.dashboard.py +++ b/grafana/risingwave-dashboard.dashboard.py @@ -524,6 +524,15 @@ def section_cluster_node(panels): ) ], ), + panels.timeseries_count( + "Meta Cluster", + "", + [ + panels.target(f"sum({metric('meta_num')}) by (worker_addr,role)", + "{{worker_addr}} @ {{role}}") + ], + ["last"], + ), ] @@ -1461,18 +1470,28 @@ def section_streaming_errors(outer_panels): panels = outer_panels.sub_panel() return [ outer_panels.row_collapsed( - "Streaming Errors", + "User Streaming Errors", [ panels.timeseries_count( - "User Errors by Type", + "Compute Errors by Type", "", [ panels.target( - f"sum({metric('user_error_count')}) by (error_type, error_msg, fragment_id, executor_name)", + f"sum({metric('user_compute_error_count')}) by (error_type, error_msg, fragment_id, executor_name)", "{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})", ), ], ), + panels.timeseries_count( + "Source Errors by Type", + "", + [ + panels.target( + f"sum({metric('user_source_error_count')}) by (error_type, error_msg, fragment_id, table_id, executor_name)", + "{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})", + ), + ], + ), ], ), ] diff --git a/grafana/risingwave-dashboard.json b/grafana/risingwave-dashboard.json index c0c19c0444b5c..53292e50fcf21 100644 --- a/grafana/risingwave-dashboard.json +++ b/grafana/risingwave-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":"Cluster Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":1},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","query":"sum(worker_num) by (worker_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":1},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"avg(process_resident_memory_bytes) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":9},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(process_cpu_seconds_total[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU","transformations":[],"transparent":false,"type":"timeseries"},{"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":17},"height":null,"hideTimeOverride":false,"id":5,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":6,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} {{source_id}} @ {{instance}}","metric":"","query":"rate(stream_source_output_rows_counts[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":7,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(partition_input_count[$__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[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":26},"height":null,"hideTimeOverride":false,"id":8,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":26},"height":null,"hideTimeOverride":false,"id":9,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(rate(partition_input_bytes[$__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[$__rate_interval]))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":34},"height":null,"hideTimeOverride":false,"id":10,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_rows_per_barrier_counts[$__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[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) per barrier","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":34},"height":null,"hideTimeOverride":false,"id":11,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all_barrier","metric":"","query":"all_barrier_nums","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"in_flight_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"in_flight_barrier","metric":"","query":"in_flight_barrier_nums","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":42},"height":null,"hideTimeOverride":false,"id":12,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_send_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count[$__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[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Send Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":42},"height":null,"hideTimeOverride":false,"id":13,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","query":"rate(meta_barrier_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":50},"height":null,"hideTimeOverride":false,"id":14,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count[$__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[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count[$__rate_interval])))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier In-Flight Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":50},"height":null,"hideTimeOverride":false,"id":15,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count[$__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[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Sync Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":58},"height":null,"hideTimeOverride":false,"id":16,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_wait_commit_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count[$__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[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Wait Commit Latency","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":66},"height":null,"hideTimeOverride":false,"id":17,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":18,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_executor_row_count[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{executor_id}}","metric":"","query":"rate(stream_executor_row_count[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":19,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_output_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_output_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Backpressure","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":20,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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":"{{actor_id}}","metric":"","query":"rate(actor_memory_usage[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Memory Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":8},"height":null,"hideTimeOverride":false,"id":21,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_input_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{upstream_fragment_id}}","metric":"","query":"rate(stream_actor_input_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":22,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_barrier_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_barrier_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":23,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_processing_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_processing_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Processing Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":24,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_actor_execution_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_actor_execution_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Execution Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":24},"height":null,"hideTimeOverride":false,"id":25,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_in_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_in_record_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":32},"height":null,"hideTimeOverride":false,"id":26,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_out_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_out_record_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":32},"height":null,"hideTimeOverride":false,"id":27,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":28,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":29,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) / rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration[$__rate_interval]) / rate(stream_actor_fast_poll_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":40},"height":null,"hideTimeOverride":false,"id":30,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":48},"height":null,"hideTimeOverride":false,"id":31,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":48},"height":null,"hideTimeOverride":false,"id":32,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) / rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration[$__rate_interval]) / rate(stream_actor_slow_poll_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":48},"height":null,"hideTimeOverride":false,"id":33,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":56},"height":null,"hideTimeOverride":false,"id":34,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":56},"height":null,"hideTimeOverride":false,"id":35,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) / rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration[$__rate_interval]) / rate(stream_actor_poll_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":56},"height":null,"hideTimeOverride":false,"id":36,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":64},"height":null,"hideTimeOverride":false,"id":37,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":64},"height":null,"hideTimeOverride":false,"id":38,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) / rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration[$__rate_interval]) / rate(stream_actor_idle_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":64},"height":null,"hideTimeOverride":false,"id":39,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":72},"height":null,"hideTimeOverride":false,"id":40,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":72},"height":null,"hideTimeOverride":false,"id":41,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) / rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration[$__rate_interval]) / rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":42,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_lookup_total_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_insert_cache_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss when insert{{actor_id}} {{side}}","metric":"","query":"rate(stream_join_insert_cache_miss_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":43,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, actor_id, wait_side, job, instance)(rate(stream_join_barrier_align_duration_sum[$__rate_interval])) / sum by(le,actor_id,wait_side,job,instance) (rate(stream_join_barrier_align_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"sum by(le, actor_id, wait_side, job, instance)(rate(stream_join_barrier_align_duration_sum[$__rate_interval])) / sum by(le,actor_id,wait_side,job,instance) (rate(stream_join_barrier_align_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Barrier Align","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":88},"height":null,"hideTimeOverride":false,"id":44,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_actor_input_waiting_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_join_actor_input_waiting_duration_ns[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":88},"height":null,"hideTimeOverride":false,"id":45,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_match_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}.{{side}}","metric":"","query":"rate(stream_join_match_duration_ns[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":96},"height":null,"hideTimeOverride":false,"id":46,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_entries","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_entries","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Entries","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":96},"height":null,"hideTimeOverride":false,"id":47,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_rows","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_rows","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":104},"height":null,"hideTimeOverride":false,"id":48,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_estimated_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_estimated_size","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Estimated Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":104},"height":null,"hideTimeOverride":false,"id":49,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_total_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each Key/State","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":112},"height":null,"hideTimeOverride":false,"id":50,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss {{actor_id}}","metric":"","query":"rate(stream_agg_chunk_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups {{actor_id}}","metric":"","query":"rate(stream_agg_chunk_lookup_total_count[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":112},"height":null,"hideTimeOverride":false,"id":51,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_cached_keys","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"stream_agg_cached_keys","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Cached Keys","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":67},"height":null,"hideTimeOverride":false,"id":52,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":53,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_send_size[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":54,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_recv_size[$__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[$__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":68},"height":null,"hideTimeOverride":false,"id":55,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":56,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_error_count) 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_error_count) by (error_type, error_msg, fragment_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"User Errors by Type","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"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":69},"height":null,"hideTimeOverride":false,"id":57,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":58,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_exchange_recv_row_number","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_task_exchange_recv_row_number","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Exchange Recv Row Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":0},"height":null,"hideTimeOverride":false,"id":59,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_num","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_task_num","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mpp Task Number","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":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":70},"height":null,"hideTimeOverride":false,"id":60,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":71},"height":null,"hideTimeOverride":false,"id":61,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_duration_bucket[$__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(state_store_sync_duration_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Build and Sync Sstable Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":71},"height":null,"hideTimeOverride":false,"id":62,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (job, instance, table_id, type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__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[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__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[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":79},"height":null,"hideTimeOverride":false,"id":63,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_duration_count[$__rate_interval])) by (job,instanc,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[$__rate_interval])) by (job,instanc,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_range_reverse_scan_duration_count[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"backward scan - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_range_reverse_scan_duration_count[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_shared_buffer_hit_counts[$__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[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":79},"height":null,"hideTimeOverride":false,"id":64,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_get_duration_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count[$__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[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":87},"height":null,"hideTimeOverride":false,"id":65,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_duration_bucket[$__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_duration_bucket[$__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_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_duration_bucket[$__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_duration_bucket[$__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_duration_bucket[$__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_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_duration_count[$__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_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_duration_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_scan_duration_bucket[$__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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_scan_duration_bucket[$__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[$__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[$__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_scan_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_scan_duration_count[$__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_scan_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_scan_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":87},"height":null,"hideTimeOverride":false,"id":66,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.9, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.9, sum(rate(state_store_get_value_size_bucket[$__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[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket[$__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[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.999, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.999, sum(rate(state_store_get_value_size_bucket[$__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[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket[$__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[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":95},"height":null,"hideTimeOverride":false,"id":67,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_size_bucket[$__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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_size_bucket[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":95},"height":null,"hideTimeOverride":false,"id":68,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_item_bucket[$__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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_item_bucket[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":103},"height":null,"hideTimeOverride":false,"id":69,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum[$__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[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":103},"height":null,"hideTimeOverride":false,"id":70,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_size_sum[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":111},"height":null,"hideTimeOverride":false,"id":71,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_may_exist_duration_bucket[$__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_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_may_exist_duration_bucket[$__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_may_exist_duration_bucket[$__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_may_exist_duration_bucket[$__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_may_exist_duration_bucket[$__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_may_exist_duration_bucket[$__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_may_exist_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_may_exist_duration_count[$__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_may_exist_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_may_exist_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - MayExist","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":111},"height":null,"hideTimeOverride":false,"id":72,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter true negative - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive count - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req bloom filter positive - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req check bloom filter - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Bloom Filter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":119},"height":null,"hideTimeOverride":false,"id":73,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":119},"height":null,"hideTimeOverride":false,"id":74,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)) / (sum(rate(state_bloom_filter_check_counts[$__rate_interval])) by (job,instance,table_id,type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter miss rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)) / (sum(rate(state_bloom_filter_check_counts[$__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'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total'}[$__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'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total'}[$__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'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total'}[$__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'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total'}[$__rate_interval])) by (job,instance,table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(file_cache_miss[$__rate_interval])) by (instance)) / (sum(rate(file_cache_latency_count{op='get'}[$__rate_interval])) by (instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss rate @ {{instance}}","metric":"","query":"(sum(rate(file_cache_miss[$__rate_interval])) by (instance)) / (sum(rate(file_cache_latency_count{op='get'}[$__rate_interval])) by (instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter filter rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (((sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter false positive rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (((sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":" Filter/Cache Miss Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":127},"height":null,"hideTimeOverride":false,"id":75,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p90 - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count[$__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[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Merged SSTs","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":127},"height":null,"hideTimeOverride":false,"id":76,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":135},"height":null,"hideTimeOverride":false,"id":77,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_write_batch_duration_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count[$__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[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count[$__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[$__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[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":135},"height":null,"hideTimeOverride":false,"id":78,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_tuple_counts[$__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(rate(state_store_write_batch_tuple_counts[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":143},"height":null,"hideTimeOverride":false,"id":79,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__rate_interval]))by(job,instance) / sum(rate(state_store_write_batch_size_count[$__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[$__rate_interval]))by(job,instance) / sum(rate(state_store_write_batch_size_count[$__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[$__rate_interval]))by(job,instance) / sum(rate(state_store_shared_buffer_to_sstable_size_count[$__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[$__rate_interval]))by(job,instance) / sum(rate(state_store_shared_buffer_to_sstable_size_count[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":143},"height":null,"hideTimeOverride":false,"id":80,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_size_bucket[$__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(state_store_sync_size_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Sync Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":151},"height":null,"hideTimeOverride":false,"id":81,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_meta_cache_size) 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) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_block_cache_size) 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) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_limit_memory_size) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading memory - {{job}}","metric":"","query":"sum(state_store_limit_memory_size) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":151},"height":null,"hideTimeOverride":false,"id":82,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(batch_row_seq_scan_next_duration_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Row SeqScan Next Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":159},"height":null,"hideTimeOverride":false,"id":83,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count[$__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[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Duration","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":167},"height":null,"hideTimeOverride":false,"id":84,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"num of SSTs in 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":85,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_sst_num) 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) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"KBs total file bytes in 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":86,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) 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) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs level sst","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compactions from each level 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":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":87,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency) 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) by (compactor, group, task_type, result)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Success & Failure Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compaction task which does not trigger","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":88,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_skip_compact_frequency[$__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[$__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":null,"description":"num of compactions from each level 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":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":89,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(storage_compact_task_pending_num) 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) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Running Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total time of compact that have been issued to 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":90,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","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[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","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[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(compactor_compact_task_duration_sum[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count[$__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[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(state_store_compact_sst_duration_sum[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count[$__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[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"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":0,"y":24},"height":null,"hideTimeOverride":false,"id":91,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by(job,instance) + sum(rate(storage_level_compact_read_curr[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by(job,instance) + sum(rate(storage_level_compact_read_curr[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(storage_level_compact_write[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_write_build_l0_bytes[$__rate_interval]))by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}} @ {{instance}}","metric":"","query":"sum(rate(compactor_write_build_l0_bytes[$__rate_interval]))by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into 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":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":92,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","query":"sum(storage_level_compact_write) by (job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","query":"sum(compactor_write_build_l0_bytes) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into 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":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":93,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) / sum(state_store_write_build_l0_bytes)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write amplification","metric":"","query":"sum(storage_level_compact_write) / sum(state_store_write_build_l0_bytes)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Amplification","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs to be merged to next level in 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":12,"y":32},"height":null,"hideTimeOverride":false,"id":94,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_cnt","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"storage_level_compact_cnt","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":40},"height":null,"hideTimeOverride":false,"id":95,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":96,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_curr[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":48},"height":null,"hideTimeOverride":false,"id":97,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_sstn_curr[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":48},"height":null,"hideTimeOverride":false,"id":98,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","query":"sum(rate(storage_level_compact_write[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":56},"height":null,"hideTimeOverride":false,"id":99,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write_sstn[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","query":"sum(rate(storage_level_compact_write_sstn[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs 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":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":100,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_sstn_next[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"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":64},"height":null,"hideTimeOverride":false,"id":101,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"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":64},"height":null,"hideTimeOverride":false,"id":102,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_sstable_avg_key_size_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_sstable_avg_key_size_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Item Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"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":0,"y":72},"height":null,"hideTimeOverride":false,"id":103,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":72},"height":null,"hideTimeOverride":false,"id":104,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_iter_scan_key_counts[$__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[$__rate_interval])) by (instance, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Iter keys","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":168},"height":null,"hideTimeOverride":false,"id":105,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":106,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes[$__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[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes[$__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[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":107,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(object_store_operation_latency_bucket[$__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[$__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[$__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[$__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[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, type)(rate(object_store_operation_latency_sum[$__rate_interval])) / sum by(le, type) (rate(object_store_operation_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} avg","metric":"","query":"sum by(le, type)(rate(object_store_operation_latency_sum[$__rate_interval])) / sum by(le, type) (rate(object_store_operation_latency_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":108,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count[$__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[$__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'}[$__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'}[$__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'}[$__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'}[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":109,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(object_store_operation_bytes_bucket[$__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[$__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[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":110,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":111,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_read_bytes) * 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) * 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'}) * 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'}) * 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'}) * 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'}) * 0.005 / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Realtime)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":112,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) 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) 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":169},"height":null,"hideTimeOverride":false,"id":113,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":114,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__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[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__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[$__rate_interval])) by (instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache disk {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_disk_latency_count[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":115,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":116,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_bytes[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"disk {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_disk_bytes[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":117,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Disk IO Size","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":170},"height":null,"hideTimeOverride":false,"id":118,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":119,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p999 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.999, sum(rate(hummock_manager_lock_time_bucket[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":120,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p999 - {{method}}","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_hummock_manager_real_process_time_bucket[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":121,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version size","metric":"","query":"storage_version_size","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":122,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"current version id","metric":"","query":"storage_current_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_checkpoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"checkpoint version id","metric":"","query":"storage_checkpoint_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned version id","metric":"","query":"storage_min_pinned_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_safepoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min safepoint version id","metric":"","query":"storage_min_safepoint_version_id","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Id","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":123,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_max_committed_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"max committed epoch","metric":"","query":"storage_max_committed_epoch","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_safe_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"safe epoch","metric":"","query":"storage_safe_epoch","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned epoch","metric":"","query":"storage_min_pinned_epoch","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":124,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_size'}/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'}/1024","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_value_size'}/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'}/1024","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":125,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_count'}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_key_count'}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"total number of SSTs that is no longer referenced by versions but is not yet deleted from 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":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":126,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_ssts_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stale SST total number","metric":"","query":"storage_stale_ssts_count","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Stale SST Total Number","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":171},"height":null,"hideTimeOverride":false,"id":127,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"backup_job_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"job count","metric":"","query":"backup_job_count","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":129,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p999 - {{state}}","metric":"","query":"histogram_quantile(0.999, sum(rate(backup_job_latency_bucket[$__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[$__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[$__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":172},"height":null,"hideTimeOverride":false,"id":130,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":131,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":132,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Drop latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":133,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog'}[$__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":173},"height":null,"hideTimeOverride":false,"id":134,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":135,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"AddWorkerNode latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":136,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes'}[$__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":174},"height":null,"hideTimeOverride":false,"id":137,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":138,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"CreateMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":139,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"DropMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":140,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush'}[$__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":175},"height":null,"hideTimeOverride":false,"id":141,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":142,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinVersionBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":143,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinSnapshotBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":144,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ReportCompactionTasks latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":145,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds'}[$__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":176},"height":null,"hideTimeOverride":false,"id":146,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":147,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":148,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count[$__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[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":149,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":150,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_counts[$__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[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_counts[$__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[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":151,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count[$__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[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":152,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":153,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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":177},"height":null,"hideTimeOverride":false,"id":154,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":0},"height":null,"hideTimeOverride":false,"id":155,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(frontend_query_counter_local_execution[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Loacl Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":156,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(distributed_completed_query_counter[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":157,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num","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","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Running query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":158,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter","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","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Rejected query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":159,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter","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","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Completed query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":160,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":161,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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":"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":178},"height":null,"hideTimeOverride":false,"id":162,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":163,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(lru_runtime_loop_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(lru_runtime_loop_count[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":164,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_watermark_step","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_watermark_step","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark steps","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":165,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_current_watermark_time_ms","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_current_watermark_time_ms","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_physical_now_ms","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_physical_now_ms","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark_time and physical_now","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_allocated_bytes","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jemalloc_allocated_bytes","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":167,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_total_mem_usage","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"stream_total_mem_usage","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by streaming","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory manager","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[]},"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":"Ecy3uV1nz","version":0} +{"__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":"Cluster Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":1},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","query":"sum(worker_num) by (worker_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":1},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"avg(process_resident_memory_bytes) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":9},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(process_cpu_seconds_total[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":9},"height":null,"hideTimeOverride":false,"id":5,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(meta_num) by (worker_addr,role)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_addr}} @ {{role}}","metric":"","query":"sum(meta_num) by (worker_addr,role)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Meta Cluster","transformations":[],"transparent":false,"type":"timeseries"},{"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":17},"height":null,"hideTimeOverride":false,"id":6,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} {{source_id}} @ {{instance}}","metric":"","query":"rate(stream_source_output_rows_counts[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(partition_input_count[$__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[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":26},"height":null,"hideTimeOverride":false,"id":9,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":26},"height":null,"hideTimeOverride":false,"id":10,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(rate(partition_input_bytes[$__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[$__rate_interval]))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":34},"height":null,"hideTimeOverride":false,"id":11,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_rows_per_barrier_counts[$__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[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) per barrier","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":34},"height":null,"hideTimeOverride":false,"id":12,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all_barrier","metric":"","query":"all_barrier_nums","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"in_flight_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"in_flight_barrier","metric":"","query":"in_flight_barrier_nums","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":42},"height":null,"hideTimeOverride":false,"id":13,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_send_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count[$__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[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Send Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":42},"height":null,"hideTimeOverride":false,"id":14,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","query":"rate(meta_barrier_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":50},"height":null,"hideTimeOverride":false,"id":15,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count[$__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[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count[$__rate_interval])))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier In-Flight Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":50},"height":null,"hideTimeOverride":false,"id":16,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count[$__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[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Sync Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":58},"height":null,"hideTimeOverride":false,"id":17,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_wait_commit_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count[$__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[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Wait Commit Latency","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":66},"height":null,"hideTimeOverride":false,"id":18,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":19,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_executor_row_count[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{executor_id}}","metric":"","query":"rate(stream_executor_row_count[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":20,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_output_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_output_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Backpressure","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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":"{{actor_id}}","metric":"","query":"rate(actor_memory_usage[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Memory Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":8},"height":null,"hideTimeOverride":false,"id":22,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_input_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{upstream_fragment_id}}","metric":"","query":"rate(stream_actor_input_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":23,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_barrier_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_barrier_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":24,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_processing_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_processing_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Processing Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":25,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_actor_execution_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_actor_execution_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Execution Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":24},"height":null,"hideTimeOverride":false,"id":26,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_in_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_in_record_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":32},"height":null,"hideTimeOverride":false,"id":27,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_out_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_out_record_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":32},"height":null,"hideTimeOverride":false,"id":28,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":29,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":30,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) / rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration[$__rate_interval]) / rate(stream_actor_fast_poll_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":40},"height":null,"hideTimeOverride":false,"id":31,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":48},"height":null,"hideTimeOverride":false,"id":32,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":48},"height":null,"hideTimeOverride":false,"id":33,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) / rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration[$__rate_interval]) / rate(stream_actor_slow_poll_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":48},"height":null,"hideTimeOverride":false,"id":34,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":56},"height":null,"hideTimeOverride":false,"id":35,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":56},"height":null,"hideTimeOverride":false,"id":36,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) / rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration[$__rate_interval]) / rate(stream_actor_poll_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":56},"height":null,"hideTimeOverride":false,"id":37,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":64},"height":null,"hideTimeOverride":false,"id":38,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":64},"height":null,"hideTimeOverride":false,"id":39,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) / rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration[$__rate_interval]) / rate(stream_actor_idle_cnt[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":64},"height":null,"hideTimeOverride":false,"id":40,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":72},"height":null,"hideTimeOverride":false,"id":41,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":72},"height":null,"hideTimeOverride":false,"id":42,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) / rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration[$__rate_interval]) / rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":43,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_lookup_total_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_insert_cache_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss when insert{{actor_id}} {{side}}","metric":"","query":"rate(stream_join_insert_cache_miss_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":44,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, actor_id, wait_side, job, instance)(rate(stream_join_barrier_align_duration_sum[$__rate_interval])) / sum by(le,actor_id,wait_side,job,instance) (rate(stream_join_barrier_align_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"sum by(le, actor_id, wait_side, job, instance)(rate(stream_join_barrier_align_duration_sum[$__rate_interval])) / sum by(le,actor_id,wait_side,job,instance) (rate(stream_join_barrier_align_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Barrier Align","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":88},"height":null,"hideTimeOverride":false,"id":45,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_actor_input_waiting_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_join_actor_input_waiting_duration_ns[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":88},"height":null,"hideTimeOverride":false,"id":46,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_match_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}.{{side}}","metric":"","query":"rate(stream_join_match_duration_ns[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":96},"height":null,"hideTimeOverride":false,"id":47,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_entries","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_entries","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Entries","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":96},"height":null,"hideTimeOverride":false,"id":48,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_rows","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_rows","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":104},"height":null,"hideTimeOverride":false,"id":49,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_estimated_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_estimated_size","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Estimated Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":104},"height":null,"hideTimeOverride":false,"id":50,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_total_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each Key/State","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":112},"height":null,"hideTimeOverride":false,"id":51,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss {{actor_id}}","metric":"","query":"rate(stream_agg_chunk_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups {{actor_id}}","metric":"","query":"rate(stream_agg_chunk_lookup_total_count[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":112},"height":null,"hideTimeOverride":false,"id":52,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_cached_keys","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"stream_agg_cached_keys","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Cached Keys","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":67},"height":null,"hideTimeOverride":false,"id":53,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":54,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_send_size[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":55,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_recv_size[$__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[$__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":68},"height":null,"hideTimeOverride":false,"id":56,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":57,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count) 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) 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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":58,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count) 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) 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"}],"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":69},"height":null,"hideTimeOverride":false,"id":59,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":60,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_exchange_recv_row_number","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_task_exchange_recv_row_number","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Exchange Recv Row Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":0},"height":null,"hideTimeOverride":false,"id":61,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_num","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_task_num","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mpp Task Number","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":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":70},"height":null,"hideTimeOverride":false,"id":62,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":71},"height":null,"hideTimeOverride":false,"id":63,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_duration_bucket[$__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(state_store_sync_duration_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Build and Sync Sstable Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":71},"height":null,"hideTimeOverride":false,"id":64,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (job, instance, table_id, type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__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[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__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[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":79},"height":null,"hideTimeOverride":false,"id":65,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_duration_count[$__rate_interval])) by (job,instanc,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[$__rate_interval])) by (job,instanc,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_range_reverse_scan_duration_count[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"backward scan - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_range_reverse_scan_duration_count[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_shared_buffer_hit_counts[$__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[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":79},"height":null,"hideTimeOverride":false,"id":66,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_get_duration_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count[$__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[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":87},"height":null,"hideTimeOverride":false,"id":67,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_duration_bucket[$__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_duration_bucket[$__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_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_duration_bucket[$__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_duration_bucket[$__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_duration_bucket[$__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_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_duration_count[$__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_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_duration_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_scan_duration_bucket[$__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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_scan_duration_bucket[$__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[$__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[$__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_scan_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_scan_duration_count[$__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_scan_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_scan_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":87},"height":null,"hideTimeOverride":false,"id":68,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.9, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.9, sum(rate(state_store_get_value_size_bucket[$__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[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket[$__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[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.999, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.999, sum(rate(state_store_get_value_size_bucket[$__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[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket[$__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[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":95},"height":null,"hideTimeOverride":false,"id":69,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_size_bucket[$__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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_size_bucket[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":95},"height":null,"hideTimeOverride":false,"id":70,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_item_bucket[$__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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_item_bucket[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":103},"height":null,"hideTimeOverride":false,"id":71,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum[$__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[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":103},"height":null,"hideTimeOverride":false,"id":72,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_size_sum[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":111},"height":null,"hideTimeOverride":false,"id":73,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_may_exist_duration_bucket[$__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_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_may_exist_duration_bucket[$__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_may_exist_duration_bucket[$__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_may_exist_duration_bucket[$__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_may_exist_duration_bucket[$__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_may_exist_duration_bucket[$__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_may_exist_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_may_exist_duration_count[$__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_may_exist_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_may_exist_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - MayExist","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":111},"height":null,"hideTimeOverride":false,"id":74,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter true negative - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive count - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req bloom filter positive - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req check bloom filter - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Bloom Filter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":119},"height":null,"hideTimeOverride":false,"id":75,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":119},"height":null,"hideTimeOverride":false,"id":76,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)) / (sum(rate(state_bloom_filter_check_counts[$__rate_interval])) by (job,instance,table_id,type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter miss rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)) / (sum(rate(state_bloom_filter_check_counts[$__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'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total'}[$__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'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total'}[$__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'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total'}[$__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'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total'}[$__rate_interval])) by (job,instance,table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(file_cache_miss[$__rate_interval])) by (instance)) / (sum(rate(file_cache_latency_count{op='get'}[$__rate_interval])) by (instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss rate @ {{instance}}","metric":"","query":"(sum(rate(file_cache_miss[$__rate_interval])) by (instance)) / (sum(rate(file_cache_latency_count{op='get'}[$__rate_interval])) by (instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter filter rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (((sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter false positive rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (((sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":" Filter/Cache Miss Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":127},"height":null,"hideTimeOverride":false,"id":77,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p90 - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count[$__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[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Merged SSTs","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":127},"height":null,"hideTimeOverride":false,"id":78,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":135},"height":null,"hideTimeOverride":false,"id":79,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_write_batch_duration_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count[$__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[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count[$__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[$__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[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":135},"height":null,"hideTimeOverride":false,"id":80,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_tuple_counts[$__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(rate(state_store_write_batch_tuple_counts[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":143},"height":null,"hideTimeOverride":false,"id":81,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__rate_interval]))by(job,instance) / sum(rate(state_store_write_batch_size_count[$__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[$__rate_interval]))by(job,instance) / sum(rate(state_store_write_batch_size_count[$__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[$__rate_interval]))by(job,instance) / sum(rate(state_store_shared_buffer_to_sstable_size_count[$__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[$__rate_interval]))by(job,instance) / sum(rate(state_store_shared_buffer_to_sstable_size_count[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":143},"height":null,"hideTimeOverride":false,"id":82,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_size_bucket[$__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(state_store_sync_size_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Sync Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":151},"height":null,"hideTimeOverride":false,"id":83,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_meta_cache_size) 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) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_block_cache_size) 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) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_limit_memory_size) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading memory - {{job}}","metric":"","query":"sum(state_store_limit_memory_size) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":151},"height":null,"hideTimeOverride":false,"id":84,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(batch_row_seq_scan_next_duration_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count[$__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[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Row SeqScan Next Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":159},"height":null,"hideTimeOverride":false,"id":85,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count[$__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[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Duration","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":167},"height":null,"hideTimeOverride":false,"id":86,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"num of SSTs in 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":87,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_sst_num) 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) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"KBs total file bytes in 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":88,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) 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) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs level sst","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compactions from each level 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":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":89,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency) 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) by (compactor, group, task_type, result)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Success & Failure Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compaction task which does not trigger","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_skip_compact_frequency[$__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[$__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":null,"description":"num of compactions from each level 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":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":91,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(storage_compact_task_pending_num) 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) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Running Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total time of compact that have been issued to 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":92,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","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[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","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[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(compactor_compact_task_duration_sum[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count[$__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[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(state_store_compact_sst_duration_sum[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count[$__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[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"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":0,"y":24},"height":null,"hideTimeOverride":false,"id":93,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by(job,instance) + sum(rate(storage_level_compact_read_curr[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by(job,instance) + sum(rate(storage_level_compact_read_curr[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(storage_level_compact_write[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_write_build_l0_bytes[$__rate_interval]))by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}} @ {{instance}}","metric":"","query":"sum(rate(compactor_write_build_l0_bytes[$__rate_interval]))by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into 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":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":94,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","query":"sum(storage_level_compact_write) by (job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","query":"sum(compactor_write_build_l0_bytes) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into 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":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":95,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) / sum(state_store_write_build_l0_bytes)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write amplification","metric":"","query":"sum(storage_level_compact_write) / sum(state_store_write_build_l0_bytes)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Amplification","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs to be merged to next level in 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":12,"y":32},"height":null,"hideTimeOverride":false,"id":96,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_cnt","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"storage_level_compact_cnt","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":40},"height":null,"hideTimeOverride":false,"id":97,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":98,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_curr[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":48},"height":null,"hideTimeOverride":false,"id":99,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_sstn_curr[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":48},"height":null,"hideTimeOverride":false,"id":100,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","query":"sum(rate(storage_level_compact_write[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":56},"height":null,"hideTimeOverride":false,"id":101,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write_sstn[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","query":"sum(rate(storage_level_compact_write_sstn[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs 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":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":102,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_sstn_next[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"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":64},"height":null,"hideTimeOverride":false,"id":103,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"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":64},"height":null,"hideTimeOverride":false,"id":104,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_sstable_avg_key_size_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_sstable_avg_key_size_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count[$__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[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Item Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"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":0,"y":72},"height":null,"hideTimeOverride":false,"id":105,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":72},"height":null,"hideTimeOverride":false,"id":106,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_iter_scan_key_counts[$__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[$__rate_interval])) by (instance, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Iter keys","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":168},"height":null,"hideTimeOverride":false,"id":107,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":108,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes[$__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[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes[$__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[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":109,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(object_store_operation_latency_bucket[$__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[$__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[$__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[$__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[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, type)(rate(object_store_operation_latency_sum[$__rate_interval])) / sum by(le, type) (rate(object_store_operation_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} avg","metric":"","query":"sum by(le, type)(rate(object_store_operation_latency_sum[$__rate_interval])) / sum by(le, type) (rate(object_store_operation_latency_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":110,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count[$__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[$__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'}[$__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'}[$__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'}[$__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'}[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":111,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(object_store_operation_bytes_bucket[$__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[$__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[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":112,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":113,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_read_bytes) * 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) * 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'}) * 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'}) * 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'}) * 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'}) * 0.005 / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Realtime)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":114,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) 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) 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":169},"height":null,"hideTimeOverride":false,"id":115,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":116,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__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[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__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[$__rate_interval])) by (instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache disk {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_disk_latency_count[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":118,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_bytes[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"disk {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_disk_bytes[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":119,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Disk IO Size","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":170},"height":null,"hideTimeOverride":false,"id":120,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":121,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p999 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.999, sum(rate(hummock_manager_lock_time_bucket[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":122,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p999 - {{method}}","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_hummock_manager_real_process_time_bucket[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":123,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version size","metric":"","query":"storage_version_size","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":124,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"current version id","metric":"","query":"storage_current_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_checkpoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"checkpoint version id","metric":"","query":"storage_checkpoint_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned version id","metric":"","query":"storage_min_pinned_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_safepoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min safepoint version id","metric":"","query":"storage_min_safepoint_version_id","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Id","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":125,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_max_committed_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"max committed epoch","metric":"","query":"storage_max_committed_epoch","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_safe_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"safe epoch","metric":"","query":"storage_safe_epoch","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned epoch","metric":"","query":"storage_min_pinned_epoch","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":126,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_size'}/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'}/1024","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_value_size'}/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'}/1024","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":127,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_count'}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_key_count'}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"total number of SSTs that is no longer referenced by versions but is not yet deleted from 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":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":128,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_ssts_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stale SST total number","metric":"","query":"storage_stale_ssts_count","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Stale SST Total Number","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":171},"height":null,"hideTimeOverride":false,"id":129,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":130,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"backup_job_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"job count","metric":"","query":"backup_job_count","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":131,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p999 - {{state}}","metric":"","query":"histogram_quantile(0.999, sum(rate(backup_job_latency_bucket[$__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[$__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[$__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":172},"height":null,"hideTimeOverride":false,"id":132,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":133,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":134,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Drop latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":135,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog'}[$__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":173},"height":null,"hideTimeOverride":false,"id":136,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":137,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"AddWorkerNode latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":138,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes'}[$__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":174},"height":null,"hideTimeOverride":false,"id":139,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":140,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"CreateMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":141,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"DropMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":142,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush'}[$__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":175},"height":null,"hideTimeOverride":false,"id":143,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":144,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinVersionBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":145,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinSnapshotBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":146,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ReportCompactionTasks latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":147,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds'}[$__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'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds'}[$__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":176},"height":null,"hideTimeOverride":false,"id":148,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":149,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":150,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count[$__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[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":151,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":152,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_counts[$__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[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_counts[$__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[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":153,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count[$__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[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":154,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":155,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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[$__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[$__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":177},"height":null,"hideTimeOverride":false,"id":156,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":0},"height":null,"hideTimeOverride":false,"id":157,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(frontend_query_counter_local_execution[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Loacl Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":158,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(distributed_completed_query_counter[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":159,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num","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","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Running query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":160,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter","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","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Rejected query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":161,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter","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","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Completed query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":162,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":163,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"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[$__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[$__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[$__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[$__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[$__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[$__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":"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":178},"height":null,"hideTimeOverride":false,"id":164,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":165,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(lru_runtime_loop_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(lru_runtime_loop_count[$__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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":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":166,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_watermark_step","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_watermark_step","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark steps","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"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":167,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_physical_now_ms - lru_current_watermark_time_ms","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_physical_now_ms - lru_current_watermark_time_ms","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":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":168,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_allocated_bytes","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jemalloc_allocated_bytes","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"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":169,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_total_mem_usage","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"stream_total_mem_usage","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by streaming","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory manager","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[]},"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":"Ecy3uV1nz","version":0} diff --git a/proto/catalog.proto b/proto/catalog.proto index 870d4562b74be..6b36025272322 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -30,6 +30,7 @@ message StreamSourceInfo { string proto_message_name = 4; int32 csv_delimiter = 5; bool csv_has_header = 6; + string upsert_avro_primary_key = 7; } message Source { diff --git a/proto/common.proto b/proto/common.proto index e9d10e51d4e4c..1cf2e8e594307 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -32,6 +32,7 @@ enum WorkerType { COMPUTE_NODE = 2; RISE_CTL = 3; COMPACTOR = 4; + META = 5; } message ParallelUnit { diff --git a/proto/expr.proto b/proto/expr.proto index 6909ccbf53e0d..89b2049dffdb4 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -120,7 +120,16 @@ message ExprNode { ARRAY_PREPEND = 533; FORMAT_TYPE = 534; - // Non-pure functions below (> 600) + // Jsonb functions + + // jsonb -> int, jsonb -> text, jsonb #> text[] that returns jsonb + JSONB_ACCESS_INNER = 600; + // jsonb ->> int, jsonb ->> text, jsonb #>> text[] that returns text + JSONB_ACCESS_STR = 601; + JSONB_TYPEOF = 602; + JSONB_ARRAY_LENGTH = 603; + + // Non-pure functions below (> 1000) // ------------------------ // Internal functions VNODE = 1101; diff --git a/proto/hummock.proto b/proto/hummock.proto index e7f84b6ac7273..bbc2af71f421b 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -19,6 +19,8 @@ message SstableInfo { uint64 total_key_count = 7; // When a SST is divided, its divide_version will increase one. uint64 divide_version = 8; + uint64 min_epoch = 9; + uint64 max_epoch = 10; } enum LevelType { diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 3b0941af36e92..d862a929d3983 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -81,4 +81,6 @@ enum RowFormatType { CSV = 7; NATIVE = 8; DEBEZIUM_AVRO = 9; + UPSERT_JSON = 10; + UPSERT_AVRO = 11; } diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index bd60dcd19ffce..36811ec46f3d6 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -553,9 +553,12 @@ enum DispatcherType { NO_SHUFFLE = 4; } +// The property of an edge in the fragment graph. +// This is essientially a "logical" version of `Dispatcher`. See the doc of `Dispatcher` for more details. message DispatchStrategy { DispatcherType type = 1; - repeated uint32 column_indices = 2; + repeated uint32 dist_key_indices = 2; + repeated uint32 output_indices = 3; } // A dispatcher redistribute messages. @@ -564,7 +567,11 @@ message Dispatcher { DispatcherType type = 1; // Indices of the columns to be used for hashing. // For dispatcher types other than HASH, this is ignored. - repeated uint32 column_indices = 2; + repeated uint32 dist_key_indices = 2; + // Indices of the columns to output. + // In most cases, this contains all columns in the input. But for some cases like MV on MV or + // schema change, we may only output a subset of the columns. + repeated uint32 output_indices = 6; // The hash mapping for consistent hash. // For dispatcher types other than HASH, this is ignored. ActorMapping hash_mapping = 3; diff --git a/risedev.yml b/risedev.yml index 0e5509d61f419..adf5537d894df 100644 --- a/risedev.yml +++ b/risedev.yml @@ -176,7 +176,6 @@ profile: - use: kafka persist-data: true - 3etcd-3meta: steps: - use: etcd diff --git a/src/batch/src/executor/source.rs b/src/batch/src/executor/source.rs index 88f8779eb5282..3e79d7ee0a583 100644 --- a/src/batch/src/executor/source.rs +++ b/src/batch/src/executor/source.rs @@ -161,7 +161,7 @@ impl SourceExecutor { Some(vec![self.split]), self.column_ids, self.metrics, - SourceInfo::new(u32::MAX, self.source_id), + SourceInfo::new(u32::MAX, self.source_id, u32::MAX), ) .await?; diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index fe34788209496..567c9e8aa7968 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -15,8 +15,8 @@ normal = ["workspace-hack"] [dependencies] anyhow = "1" -arrow-array = "31" -arrow-schema = "31" +arrow-array = "33" +arrow-schema = "33" async-trait = "0.1" auto_enums = "0.7" bitflags = "1.3.2" @@ -28,6 +28,7 @@ clap = { version = "3", features = ["derive"] } comfy-table = "6" crc32fast = "1" derivative = "2" +easy-ext = "1" fixedbitset = { version = "0.4", features = ["std"] } futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = "0.2" diff --git a/src/common/src/array/column_proto_readers.rs b/src/common/src/array/column_proto_readers.rs index 3ee32e7443067..f1e85f2e40d22 100644 --- a/src/common/src/array/column_proto_readers.rs +++ b/src/common/src/array/column_proto_readers.rs @@ -55,6 +55,8 @@ pub fn read_numeric_array> } } let arr = builder.finish(); + ensure_eq!(arr.len(), cardinality); + Ok(arr.into()) } @@ -68,7 +70,7 @@ pub fn read_bool_array(array: &ProstArray, cardinality: usize) -> ArrayResult>( } } let arr = builder.finish(); + ensure_eq!(arr.len(), cardinality); + Ok(arr.into()) } diff --git a/src/common/src/array/decimal_array.rs b/src/common/src/array/decimal_array.rs index 496cb5802cc0a..cf64149d9d929 100644 --- a/src/common/src/array/decimal_array.rs +++ b/src/common/src/array/decimal_array.rs @@ -60,7 +60,7 @@ mod tests { assert_eq!(prost_array.values.len(), 1); - let decoded_array = ArrayImpl::from_protobuf(&prost_array, 4) + let decoded_array = ArrayImpl::from_protobuf(&prost_array, 8) .unwrap() .into_decimal(); diff --git a/src/common/src/array/jsonb_array.rs b/src/common/src/array/jsonb_array.rs index 09ba62b11e554..628971d571207 100644 --- a/src/common/src/array/jsonb_array.rs +++ b/src/common/src/array/jsonb_array.rs @@ -173,6 +173,10 @@ impl JsonbRef<'_> { output.freeze().into() } + pub fn is_jsonb_null(&self) -> bool { + matches!(self.0, Value::Null) + } + pub fn type_name(&self) -> &'static str { match self.0 { Value::Null => "null", @@ -184,6 +188,16 @@ impl JsonbRef<'_> { } } + pub fn array_len(&self) -> Result { + match self.0 { + Value::Array(v) => Ok(v.len()), + _ => Err(format!( + "cannot get array length of a jsonb {}", + self.type_name() + )), + } + } + pub fn as_bool(&self) -> Result { match self.0 { Value::Bool(v) => Ok(*v), @@ -207,6 +221,32 @@ impl JsonbRef<'_> { )), } } + + /// This is part of the `->>` or `#>>` syntax to access a child as string. + /// + /// * It is not `as_str`, because there is no runtime error when the jsonb type is not string. + /// * It is not same as [`Display`] or [`ToText`] (cast to string) in the following 2 cases: + /// * Jsonb null is displayed as 4-letter `null` but treated as sql null here. + /// * This function writes nothing and the caller is responsible for checking + /// [`is_jsonb_null`] to differentiate it from an empty string. + /// * Jsonb string is displayed with quotes but treated as its inner value here. + pub fn force_str(&self, writer: &mut W) -> std::fmt::Result { + match self.0 { + Value::String(v) => writer.write_str(v), + Value::Null => Ok(()), + Value::Bool(_) | Value::Number(_) | Value::Array(_) | Value::Object(_) => { + write!(writer, "{}", self.0) + } + } + } + + pub fn access_object_field(&self, field: &str) -> Option { + self.0.get(field).map(Self) + } + + pub fn access_array_element(&self, idx: usize) -> Option { + self.0.get(idx).map(Self) + } } #[derive(Debug)] diff --git a/src/common/src/array/list_array.rs b/src/common/src/array/list_array.rs index 1469548e9d83f..1c060b92c8c5e 100644 --- a/src/common/src/array/list_array.rs +++ b/src/common/src/array/list_array.rs @@ -32,8 +32,6 @@ use crate::types::{ DatumRef, Scalar, ScalarRefImpl, ToDatumRef, }; -/// This is a naive implementation of list array. -/// We will eventually move to a more efficient flatten implementation. #[derive(Debug)] pub struct ListArrayBuilder { bitmap: BitmapBuilder, @@ -146,8 +144,15 @@ impl ListArrayBuilder { } } -/// This is a naive implementation of list array. -/// We will eventually move to a more efficient flatten implementation. +/// Each item of this `ListArray` is a `List`, or called `T[]` (T array). +/// +/// * As other arrays, there is a null bitmap, with `1` meaning nonnull and `0` meaning null. +/// * As [`BytesArray`], there is an offsets `Vec` and a value `Array`. The value `Array` has all +/// items concatenated, and the offsets `Vec` stores start and end indices into it for slicing. +/// Effectively, the inner array is the flattened form, and `offsets.len() == n + 1`. +/// +/// For example, `values (array[1]), (array[]::int[]), (null), (array[2, 3]);` stores an inner +/// `I32Array` with `[1, 2, 3]`, along with offsets `[0, 1, 1, 1, 3]` and null bitmap `TTFT`. #[derive(Debug, Clone, PartialEq)] pub struct ListArray { bitmap: Bitmap, @@ -221,7 +226,11 @@ impl ListArray { ); let bitmap: Bitmap = array.get_null_bitmap()?.into(); let array_data = array.get_list_array_data()?.to_owned(); - let value = ArrayImpl::from_protobuf(array_data.value.as_ref().unwrap(), bitmap.len())?; + let flatten_len = match array_data.offsets.last() { + Some(&n) => n as usize, + None => bail!("Must have at least one element in offsets"), + }; + let value = ArrayImpl::from_protobuf(array_data.value.as_ref().unwrap(), flatten_len)?; let arr = ListArray { bitmap, offsets: array_data.offsets, diff --git a/src/common/src/config.rs b/src/common/src/config.rs index ff434c809e501..90123abc0974a 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -159,6 +159,10 @@ pub struct MetaConfig { /// Schedule space_reclaim compaction for all compaction groups with this interval. #[serde(default = "default::meta::periodic_space_reclaim_compaction_interval_sec")] pub periodic_space_reclaim_compaction_interval_sec: u64, + + /// Schedule ttl_reclaim compaction for all compaction groups with this interval. + #[serde(default = "default::meta::periodic_ttl_reclaim_compaction_interval_sec")] + pub periodic_ttl_reclaim_compaction_interval_sec: u64, } impl Default for MetaConfig { @@ -248,6 +252,10 @@ pub struct StreamingConfig { #[serde(default)] pub developer: DeveloperConfig, + + /// Max unique user stream errors per actor + #[serde(default = "default::streaming::unique_user_stream_errors")] + pub unique_user_stream_errors: usize, } impl Default for StreamingConfig { @@ -345,10 +353,6 @@ pub struct StorageConfig { #[serde(default = "default::storage::max_concurrent_compaction_task_number")] pub max_concurrent_compaction_task_number: u64, - - /// Whether to enable state_store_v1 for hummock - #[serde(default = "default::storage::enable_state_store_v1")] - pub enable_state_store_v1: bool, } impl Default for StorageConfig { @@ -507,6 +511,10 @@ mod default { pub fn periodic_space_reclaim_compaction_interval_sec() -> u64 { 3600 // 60min } + + pub fn periodic_ttl_reclaim_compaction_interval_sec() -> u64 { + 1800 // 30mi + } } pub mod server { @@ -606,10 +614,6 @@ mod default { pub fn max_concurrent_compaction_task_number() -> u64 { 16 } - - pub fn enable_state_store_v1() -> bool { - false - } } pub mod streaming { @@ -636,6 +640,10 @@ mod default { pub fn async_stack_trace() -> AsyncStackTraceOption { AsyncStackTraceOption::On } + + pub fn unique_user_stream_errors() -> usize { + 10 + } } pub mod file_cache { diff --git a/src/common/src/error.rs b/src/common/src/error.rs index afb768e7ed25d..df25081431370 100644 --- a/src/common/src/error.rs +++ b/src/common/src/error.rs @@ -13,9 +13,11 @@ // limitations under the License. use std::backtrace::Backtrace; +use std::collections::HashSet; use std::convert::Infallible; use std::fmt::{Debug, Display, Formatter}; use std::io::Error as IoError; +use std::time::{Duration, SystemTime}; use memcomparable::Error as MemComparableError; use risingwave_pb::ProstFieldNotFound; @@ -29,6 +31,8 @@ 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; pub type BoxedError = Box; @@ -422,6 +426,42 @@ macro_rules! bail { }; } +#[derive(Debug)] +pub struct ErrorSuppressor { + max_unique: usize, + unique: HashSet, + last_reset_time: SystemTime, +} + +impl ErrorSuppressor { + pub fn new(max_unique: usize) -> Self { + Self { + max_unique, + last_reset_time: SystemTime::now(), + unique: Default::default(), + } + } + + pub fn suppress_error(&mut self, error: &str) -> bool { + self.try_reset(); + if self.unique.contains(error) { + false + } else if self.unique.len() < self.max_unique { + self.unique.insert(error.to_string()); + false + } else { + // We have exceeded the capacity. + true + } + } + + fn try_reset(&mut self) { + if self.last_reset_time.elapsed().unwrap() >= ERROR_SUPPRESSOR_RESET_DURATION { + *self = Self::new(self.max_unique) + } + } +} + #[cfg(test)] mod tests { use std::convert::Into; diff --git a/src/common/src/hash/consistent_hash/bitmap.rs b/src/common/src/hash/consistent_hash/bitmap.rs new file mode 100644 index 0000000000000..c007138feb76f --- /dev/null +++ b/src/common/src/hash/consistent_hash/bitmap.rs @@ -0,0 +1,34 @@ +// 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::ops::RangeInclusive; + +use crate::buffer::Bitmap; +use crate::hash::VirtualNode; + +/// An extension trait for `Bitmap` to support virtual node operations. +#[easy_ext::ext(VnodeBitmapExt)] +impl Bitmap { + /// Enumerates the virtual nodes set to 1 in the bitmap. + pub fn iter_vnodes(&self) -> impl Iterator + '_ { + self.iter_ones().map(VirtualNode::from_index) + } + + /// Returns an iterator which yields the position ranges of continuous virtual nodes set to 1 in + /// the bitmap. + pub fn vnode_ranges(&self) -> impl Iterator> + '_ { + self.high_ranges() + .map(|r| (VirtualNode::from_index(*r.start())..=VirtualNode::from_index(*r.end()))) + } +} diff --git a/src/common/src/hash/consistent_hash/mapping.rs b/src/common/src/hash/consistent_hash/mapping.rs index b9b9f467da3b1..159c50cca8802 100644 --- a/src/common/src/hash/consistent_hash/mapping.rs +++ b/src/common/src/hash/consistent_hash/mapping.rs @@ -22,6 +22,7 @@ use itertools::Itertools; use risingwave_pb::common::{ParallelUnit, ParallelUnitMapping as ParallelUnitMappingProto}; use risingwave_pb::stream_plan::ActorMapping as ActorMappingProto; +use super::bitmap::VnodeBitmapExt; use super::vnode::{ParallelUnitId, VirtualNode}; use crate::buffer::{Bitmap, BitmapBuilder}; use crate::util::compress::compress_data; @@ -122,9 +123,9 @@ impl VnodeMapping { /// Returns `None` if the no virtual node is set in the bitmap. pub fn get_matched(&self, bitmap: &Bitmap) -> Option { bitmap - .iter_ones() + .iter_vnodes() .next() // only need to check the first one - .map(|i| self.get(VirtualNode::from_index(i))) + .map(|v| self.get(v)) } /// Iterate over all items in this mapping, in the order of vnodes. diff --git a/src/common/src/hash/consistent_hash/mod.rs b/src/common/src/hash/consistent_hash/mod.rs index 85339419f6e4e..02d077b077dcc 100644 --- a/src/common/src/hash/consistent_hash/mod.rs +++ b/src/common/src/hash/consistent_hash/mod.rs @@ -12,5 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod bitmap; pub mod mapping; pub mod vnode; diff --git a/src/common/src/hash/mod.rs b/src/common/src/hash/mod.rs index 91f3e917b3483..58dbada538a7d 100644 --- a/src/common/src/hash/mod.rs +++ b/src/common/src/hash/mod.rs @@ -16,6 +16,7 @@ mod consistent_hash; // TODO: move this to a separate module mod dispatcher; mod key; +pub use consistent_hash::bitmap::*; pub use consistent_hash::mapping::*; pub use consistent_hash::vnode::*; pub use dispatcher::HashKeyDispatcher; diff --git a/src/common/src/system_param.rs b/src/common/src/system_param/mod.rs similarity index 99% rename from src/common/src/system_param.rs rename to src/common/src/system_param/mod.rs index d2d0e57f12bc1..c525a3ae71a5c 100644 --- a/src/common/src/system_param.rs +++ b/src/common/src/system_param/mod.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod reader; + use std::collections::HashSet; use std::fmt::Debug; use std::ops::RangeBounds; diff --git a/src/common/src/system_param/reader.rs b/src/common/src/system_param/reader.rs new file mode 100644 index 0000000000000..fd4b58fc81a2d --- /dev/null +++ b/src/common/src/system_param/reader.rs @@ -0,0 +1,82 @@ +// 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_pb::meta::SystemParams as ProstSystemParams; +use tracing::warn; + +use super::system_params_to_kv; + +/// A wrapper for [`risingwave_pb::meta::SystemParams`] for 2 purposes: +/// - Avoid misuse of deprecated fields by hiding their getters. +/// - Abstract fallback logic for fields that might not be provided by meta service due to backward +/// compatibility. +#[derive(Clone, Debug)] +pub struct SystemParamsReader { + prost: ProstSystemParams, +} + +impl From for SystemParamsReader { + fn from(prost: ProstSystemParams) -> Self { + Self { prost } + } +} + +impl SystemParamsReader { + pub fn barrier_interval_ms(&self) -> u32 { + self.prost.barrier_interval_ms.unwrap() + } + + pub fn checkpoint_frequency(&self) -> u64 { + self.prost.checkpoint_frequency.unwrap() + } + + pub fn sstable_size_mb(&self) -> u32 { + self.prost.sstable_size_mb.unwrap() + } + + pub fn block_size_kb(&self) -> u32 { + self.prost.block_size_kb.unwrap() + } + + pub fn bloom_false_positive(&self) -> f64 { + self.prost.bloom_false_positive.unwrap() + } + + // TODO(zhidong): Only read from system params in v0.1.18. + pub fn state_store(&self, from_local: String) -> String { + let from_prost = self.prost.state_store.as_ref().unwrap(); + if from_prost.is_empty() { + warn!("--state-store is not specified on meta node, reading from CLI instead"); + from_local + } else { + from_prost.clone() + } + } + + pub fn data_directory(&self) -> &str { + self.prost.data_directory.as_ref().unwrap() + } + + pub fn backup_storage_url(&self) -> &str { + self.prost.backup_storage_url.as_ref().unwrap() + } + + pub fn backup_storage_directory(&self) -> &str { + self.prost.backup_storage_directory.as_ref().unwrap() + } + + pub fn to_kv(&self) -> Vec<(String, String)> { + system_params_to_kv(&self.prost).unwrap() + } +} diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 4c4196df2f15b..dacad21fbad89 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -22,6 +22,7 @@ aws-sdk-kinesis = { workspace = true } aws-sdk-s3 = { workspace = true } aws-smithy-http = { workspace = true } aws-types = { workspace = true } +bincode = "1" byteorder = "1" bytes = { version = "1", features = ["serde"] } chrono = { version = "0.4", default-features = false, features = ["clock", "std"] } @@ -66,7 +67,6 @@ tonic = { version = "0.2", package = "madsim-tonic" } tracing = "0.1" url = "2" urlencoding = "2" - [target.'cfg(not(madsim))'.dependencies] workspace-hack = { path = "../workspace-hack" } diff --git a/src/connector/src/common.rs b/src/connector/src/common.rs index a27db4c043f9f..c27ec551b7dac 100644 --- a/src/connector/src/common.rs +++ b/src/connector/src/common.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::borrow::Cow; + use aws_sdk_kinesis::Client as KinesisClient; use http::Uri; use rdkafka::ClientConfig; @@ -199,3 +201,11 @@ impl KinesisCommon { Ok(KinesisClient::from_conf(builder.build())) } } + +#[derive(Debug, Serialize, Deserialize)] +pub struct UpsertMessage<'a> { + #[serde(borrow)] + pub primary_key: Cow<'a, [u8]>, + #[serde(borrow)] + pub record: Cow<'a, [u8]>, +} diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index f369bf1ef9845..e3b0a9515654e 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -182,6 +182,7 @@ macro_rules! impl_common_parser_logic { if let Err(e) = self.parse_inner(content.as_ref(), builder.row_writer()) .await { + self.error_ctx.report_stream_source_error(&e); tracing::warn!("message parsing failed {}, skipping", e.to_string()); continue; } @@ -241,6 +242,11 @@ macro_rules! impl_common_split_reader_logic { let source_id = self.source_info.source_id.to_string(); let split_id = self.split_id.clone(); let metrics = self.metrics.clone(); + let error_ctx = $crate::source::base::SourceErrorContext::new( + self.source_info.source_id.table_id, + self.source_info.fragment_id, + self.metrics.clone() + ); let data_stream = self.into_data_stream(); @@ -265,7 +271,7 @@ macro_rules! impl_common_split_reader_logic { }) .boxed(); let parser = - $crate::parser::ByteStreamSourceParserImpl::create(parser_config)?; + $crate::parser::ByteStreamSourceParserImpl::create(parser_config, error_ctx)?; #[for_await] for msg_batch in parser.into_stream(data_stream) { yield msg_batch?; diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 25e41e2cca7a3..36665359a06a2 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -26,26 +26,32 @@ use url::Url; use super::schema_resolver::*; use super::util::{extract_inner_field_schema, from_avro_value}; +use crate::common::UpsertMessage; use crate::impl_common_parser_logic; use crate::parser::avro::util::avro_field_to_column_desc; use crate::parser::schema_registry::{extract_schema_id, Client}; use crate::parser::util::get_kafka_topic; use crate::parser::{SourceStreamChunkRowWriter, WriteGuard}; -use crate::source::SourceColumnDesc; +use crate::source::{SourceColumnDesc, SourceErrorContext}; impl_common_parser_logic!(AvroParser); #[derive(Debug)] pub struct AvroParser { schema: Arc, + key_schema: Option>, schema_resolver: Option>, rw_columns: Vec, + error_ctx: SourceErrorContext, + upsert_primary_key_column_name: Option, } #[derive(Debug, Clone)] pub struct AvroParserConfig { pub schema: Arc, + pub key_schema: Option>, pub schema_resolver: Option>, + pub upsert_primary_key_column_name: Option, } impl AvroParserConfig { @@ -53,18 +59,39 @@ impl AvroParserConfig { props: &HashMap, schema_location: &str, use_schema_registry: bool, + enable_upsert: bool, + upsert_primary_key_column_name: Option, ) -> Result { let url = Url::parse(schema_location).map_err(|e| { InternalError(format!("failed to parse url ({}): {}", schema_location, e)) })?; - let (schema, schema_resolver) = if use_schema_registry { + if use_schema_registry { let kafka_topic = get_kafka_topic(props)?; let client = Client::new(url, props)?; - let (schema, resolver) = - ConfluentSchemaResolver::new(format!("{}-value", kafka_topic).as_str(), client) - .await?; - (Arc::new(schema), Some(Arc::new(resolver))) + let resolver = ConfluentSchemaResolver::new(client); + + Ok(Self { + schema: resolver + .get_by_subject_name(&format!("{}-value", kafka_topic)) + .await?, + key_schema: if enable_upsert { + Some( + resolver + .get_by_subject_name(&format!("{}-key", kafka_topic)) + .await?, + ) + } else { + None + }, + schema_resolver: Some(Arc::new(resolver)), + upsert_primary_key_column_name, + }) } else { + if enable_upsert { + return Err(RwError::from(InternalError( + "avro upsert without schema registry is not supported".to_string(), + ))); + } let schema_content = match url.scheme() { "file" => read_schema_from_local(url.path()), "s3" => read_schema_from_s3(&url, props).await, @@ -77,12 +104,28 @@ impl AvroParserConfig { let schema = Schema::parse_str(&schema_content).map_err(|e| { RwError::from(InternalError(format!("Avro schema parse error {}", e))) })?; - (Arc::new(schema), None) - }; - Ok(Self { - schema, - schema_resolver, - }) + Ok(Self { + schema: Arc::new(schema), + key_schema: None, + schema_resolver: None, + upsert_primary_key_column_name: None, + }) + } + } + + pub fn extract_pks(&self) -> Result> { + if let Some(Schema::Record { fields, .. }) = self.key_schema.as_deref() { + let mut index = 0; + let fields = fields + .iter() + .map(|field| avro_field_to_column_desc(&field.name, &field.schema, &mut index)) + .collect::>>()?; + Ok(fields) + } else { + Err(RwError::from(InternalError( + "schema invalid, record required".into(), + ))) + } } pub fn map_to_columns(&self) -> Result> { @@ -104,32 +147,72 @@ impl AvroParserConfig { // confluent_wire_format, kafka only, subject-name: "${topic-name}-value" impl AvroParser { - pub fn new(rw_columns: Vec, config: AvroParserConfig) -> Result { + pub fn new( + rw_columns: Vec, + config: AvroParserConfig, + error_ctx: SourceErrorContext, + ) -> Result { let AvroParserConfig { schema, + key_schema, schema_resolver, + upsert_primary_key_column_name, } = config; Ok(Self { schema, + key_schema, schema_resolver, rw_columns, + error_ctx, + upsert_primary_key_column_name, }) } + /// The presence of a `key_schema` implies that upsert is enabled. + fn is_enable_upsert(&self) -> bool { + self.key_schema.is_some() + } + pub(crate) async fn parse_inner( &self, payload: &[u8], mut writer: SourceStreamChunkRowWriter<'_>, ) -> Result { + enum Op { + Insert, + Delete, + } + + let (_payload, op) = if self.is_enable_upsert() { + let msg: UpsertMessage<'_> = bincode::deserialize(payload).map_err(|e| { + RwError::from(ProtocolError(format!( + "extract payload err {:?}, you may need to check the 'upsert' parameter", + e + ))) + })?; + if !msg.record.is_empty() { + (msg.record, Op::Insert) + } else { + (msg.primary_key, Op::Delete) + } + } else { + (payload.into(), Op::Insert) + }; + // parse payload to avro value // if use confluent schema, get writer schema from confluent schema registry let avro_value = if let Some(resolver) = &self.schema_resolver { - let (schema_id, mut raw_payload) = extract_schema_id(payload)?; + let (schema_id, mut raw_payload) = extract_schema_id(&_payload)?; let writer_schema = resolver.get(schema_id).await?; - from_avro_datum(writer_schema.as_ref(), &mut raw_payload, Some(&self.schema)) + let reader_schema = if matches!(op, Op::Delete) { + self.key_schema.as_deref() + } else { + Some(&*self.schema) + }; + from_avro_datum(writer_schema.as_ref(), &mut raw_payload, reader_schema) .map_err(|e| RwError::from(ProtocolError(e.to_string())))? } else { - let mut reader = Reader::with_schema(&self.schema, payload) + let mut reader = Reader::with_schema(&self.schema, payload as &[u8]) .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; match reader.next() { Some(Ok(v)) => v, @@ -142,18 +225,14 @@ impl AvroParser { } }; - // parse the value to rw value + // the avro can be a key or a value if let Value::Record(fields) = avro_value { - writer.insert(|column| { - let tuple = fields - .iter() - .find(|val| column.name.eq(&val.0)) - .ok_or_else(|| { - RwError::from(InternalError(format!( - "no field named {} in avro msg", - column.name - ))) - })?; + let fill = |column: &SourceColumnDesc| { + let tuple = match fields.iter().find(|val| column.name.eq(&val.0)) { + None => return Ok(None), + Some(tup) => tup, + }; + let field_schema = extract_inner_field_schema(&self.schema, Some(&column.name))?; from_avro_value(tuple.1.clone(), field_schema).map_err(|e| { tracing::error!( @@ -163,6 +242,41 @@ impl AvroParser { ); e }) + }; + match op { + Op::Insert => writer.insert(fill), + Op::Delete => writer.delete(fill), + } + } else if self.upsert_primary_key_column_name.is_some() + && matches!(op, Op::Delete) + && matches!( + avro_value, + Value::Boolean(_) + | Value::String(_) + | Value::Int(_) + | Value::Long(_) + | Value::Float(_) + | Value::Decimal(_) + | Value::Date(_) + | Value::TimestampMillis(_) + | Value::TimestampMicros(_) + | Value::Duration(_) + ) + { + writer.delete(|desc| { + if &desc.name != self.upsert_primary_key_column_name.as_ref().unwrap() { + Ok(None) + } else { + from_avro_value(avro_value.clone(), self.key_schema.as_deref().unwrap()) + .map_err(|e| { + tracing::error!( + "failed to process value ({}): {}", + String::from_utf8_lossy(payload), + e + ); + e + }) + } }) } else { Err(RwError::from(ProtocolError( @@ -196,7 +310,7 @@ mod test { }; use crate::parser::avro::util::unix_epoch_days; use crate::parser::SourceStreamChunkBuilder; - use crate::source::SourceColumnDesc; + use crate::source::{SourceColumnDesc, SourceErrorContext}; fn test_data_path(file_name: &str) -> String { let curr_dir = env::current_dir().unwrap().into_os_string(); @@ -249,12 +363,12 @@ mod test { async fn new_avro_conf_from_local(file_name: &str) -> error::Result { let schema_path = "file://".to_owned() + &test_data_path(file_name); - AvroParserConfig::new(&HashMap::new(), schema_path.as_str(), false).await + AvroParserConfig::new(&HashMap::new(), schema_path.as_str(), false, false, None).await } async fn new_avro_parser_from_local(file_name: &str) -> error::Result { let conf = new_avro_conf_from_local(file_name).await?; - AvroParser::new(Vec::default(), conf) + AvroParser::new(Vec::default(), conf, SourceErrorContext::for_test()) } #[tokio::test] diff --git a/src/connector/src/parser/avro/schema_resolver.rs b/src/connector/src/parser/avro/schema_resolver.rs index b800cdedaa482..b14746737f8c4 100644 --- a/src/connector/src/parser/avro/schema_resolver.rs +++ b/src/connector/src/parser/avro/schema_resolver.rs @@ -23,7 +23,7 @@ use risingwave_common::error::{Result, RwError}; use url::Url; use crate::aws_utils::{default_conn_config, s3_client, AwsConfigV2}; -use crate::parser::schema_registry::Client; +use crate::parser::schema_registry::{Client, ConfluentSchema}; use crate::parser::util::download_from_http; const AVRO_SCHEMA_LOCATION_S3_REGION: &str = "region"; @@ -89,20 +89,30 @@ pub struct ConfluentSchemaResolver { } impl ConfluentSchemaResolver { - // return the reader schema and a new `SchemaResolver` - pub async fn new(subject_name: &str, client: Client) -> Result<(Schema, Self)> { - let raw_schema = client.get_schema_by_subject(subject_name).await?; + async fn parse_and_cache_schema(&self, raw_schema: ConfluentSchema) -> Result> { let schema = Schema::parse_str(&raw_schema.content) .map_err(|e| RwError::from(ProtocolError(format!("Avro schema parse error {}", e))))?; - let resolver = ConfluentSchemaResolver { + let schema = Arc::new(schema); + self.writer_schemas + .insert(raw_schema.id, Arc::clone(&schema)) + .await; + Ok(schema) + } + + /// Create a new `ConfluentSchemaResolver` + pub fn new(client: Client) -> Self { + ConfluentSchemaResolver { writer_schemas: Cache::new(u64::MAX), confluent_client: client, - }; - resolver - .writer_schemas - .insert(raw_schema.id, Arc::new(schema.clone())) - .await; - Ok((schema, resolver)) + } + } + + pub async fn get_by_subject_name(&self, subject_name: &str) -> Result> { + let raw_schema = self + .confluent_client + .get_schema_by_subject(subject_name) + .await?; + self.parse_and_cache_schema(raw_schema).await } // get the writer schema by id @@ -111,15 +121,7 @@ impl ConfluentSchemaResolver { Ok(schema) } else { let raw_schema = self.confluent_client.get_schema_by_id(schema_id).await?; - - let schema = Schema::parse_str(&raw_schema.content).map_err(|e| { - RwError::from(ProtocolError(format!("Avro schema parse error {}", e))) - })?; - let schema = Arc::new(schema); - self.writer_schemas - .insert(schema_id, Arc::clone(&schema)) - .await; - Ok(schema) + self.parse_and_cache_schema(raw_schema).await } } } diff --git a/src/connector/src/parser/canal/simd_json_parser.rs b/src/connector/src/parser/canal/simd_json_parser.rs index d5f5a360cc024..188bb6e0ac51d 100644 --- a/src/connector/src/parser/canal/simd_json_parser.rs +++ b/src/connector/src/parser/canal/simd_json_parser.rs @@ -28,7 +28,7 @@ use simd_json::{BorrowedValue, StaticNode, ValueAccess}; use crate::parser::canal::operators::*; use crate::parser::util::at_least_one_ok; use crate::parser::{SourceStreamChunkRowWriter, WriteGuard}; -use crate::source::SourceColumnDesc; +use crate::source::{SourceColumnDesc, SourceErrorContext}; use crate::{ensure_rust_type, ensure_str, impl_common_parser_logic}; const AFTER: &str = "data"; @@ -40,11 +40,15 @@ impl_common_parser_logic!(CanalJsonParser); #[derive(Debug)] pub struct CanalJsonParser { pub(crate) rw_columns: Vec, + error_ctx: SourceErrorContext, } impl CanalJsonParser { - pub fn new(rw_columns: Vec) -> Result { - Ok(Self { rw_columns }) + pub fn new(rw_columns: Vec, error_ctx: SourceErrorContext) -> Result { + Ok(Self { + rw_columns, + error_ctx, + }) } #[allow(clippy::unused_async)] @@ -255,7 +259,7 @@ mod tests { SourceColumnDesc::simple("win_rate", DataType::Float64, 5.into()), ]; - let parser = CanalJsonParser::new(descs.clone()).unwrap(); + let parser = CanalJsonParser::new(descs.clone(), SourceErrorContext::for_test()).unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 2); @@ -332,7 +336,7 @@ mod tests { SourceColumnDesc::simple("v2", DataType::Int32, 1.into()), ]; - let parser = CanalJsonParser::new(descs.clone()).unwrap(); + let parser = CanalJsonParser::new(descs.clone(), SourceErrorContext::for_test()).unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 2); diff --git a/src/connector/src/parser/csv_parser.rs b/src/connector/src/parser/csv_parser.rs index a113f1d7f49fa..1de562543ef67 100644 --- a/src/connector/src/parser/csv_parser.rs +++ b/src/connector/src/parser/csv_parser.rs @@ -28,7 +28,7 @@ use crate::parser::{ BoxSourceWithStateStream, ByteStreamSourceParser, SourceColumnDesc, SourceStreamChunkBuilder, SourceStreamChunkRowWriter, StreamChunkWithState, WriteGuard, }; -use crate::source::{BoxSourceStream, SplitId}; +use crate::source::{BoxSourceStream, SourceErrorContext, SplitId}; macro_rules! to_rust_type { ($v:ident, $t:ty) => { @@ -53,10 +53,15 @@ pub struct CsvParser { output_cursor: usize, ends: Vec, ends_cursor: usize, + error_ctx: SourceErrorContext, } impl CsvParser { - pub fn new(rw_columns: Vec, parser_config: CsvParserConfig) -> Result { + pub fn new( + rw_columns: Vec, + parser_config: CsvParserConfig, + error_ctx: SourceErrorContext, + ) -> Result { let CsvParserConfig { delimiter, has_header, @@ -70,6 +75,7 @@ impl CsvParser { output_cursor: 0, ends: vec![0], ends_cursor: 1, + error_ctx, }) } @@ -316,7 +322,7 @@ mod tests { delimiter: b',', has_header: true, }; - let parser = CsvParser::new(descs, config).unwrap(); + let parser = CsvParser::new(descs, config, SourceErrorContext::for_test()).unwrap(); let data = b" name,age pite,20 @@ -341,7 +347,7 @@ alex,10"; delimiter: b',', has_header: true, }; - let parser = CsvParser::new(descs, config).unwrap(); + let parser = CsvParser::new(descs, config, SourceErrorContext::for_test()).unwrap(); let data = b" name,age pite,20 diff --git a/src/connector/src/parser/debezium/avro_parser.rs b/src/connector/src/parser/debezium/avro_parser.rs index bf524192dcdaf..036a2b895b2df 100644 --- a/src/connector/src/parser/debezium/avro_parser.rs +++ b/src/connector/src/parser/debezium/avro_parser.rs @@ -35,7 +35,7 @@ use crate::parser::schema_registry::{extract_schema_id, Client}; use crate::parser::schema_resolver::ConfluentSchemaResolver; use crate::parser::util::get_kafka_topic; use crate::parser::{SourceStreamChunkRowWriter, WriteGuard}; -use crate::source::SourceColumnDesc; +use crate::source::{SourceColumnDesc, SourceErrorContext}; const BEFORE: &str = "before"; const AFTER: &str = "after"; @@ -51,6 +51,7 @@ pub struct DebeziumAvroParser { inner_schema: Arc, schema_resolver: Arc, rw_columns: Vec, + error_ctx: SourceErrorContext, } #[derive(Debug, Clone)] @@ -75,12 +76,14 @@ impl DebeziumAvroParserConfig { let key_schema = Schema::parse_str(&raw_schema.content) .map_err(|e| RwError::from(ProtocolError(format!("Avro schema parse error {}", e))))?; - let (outer_schema, resolver) = - ConfluentSchemaResolver::new(format!("{}-value", kafka_topic).as_str(), client).await?; + let resolver = ConfluentSchemaResolver::new(client); + let outer_schema = resolver + .get_by_subject_name(&format!("{}-value", kafka_topic)) + .await?; let inner_schema = Self::extract_inner_schema(&outer_schema)?; Ok(Self { key_schema: Arc::new(key_schema), - outer_schema: Arc::new(outer_schema), + outer_schema, inner_schema: Arc::new(inner_schema), schema_resolver: Arc::new(resolver), }) @@ -167,6 +170,7 @@ impl DebeziumAvroParser { pub fn new( rw_columns: Vec, config: DebeziumAvroParserConfig, + error_ctx: SourceErrorContext, ) -> Result { let DebeziumAvroParserConfig { outer_schema, @@ -179,6 +183,7 @@ impl DebeziumAvroParser { inner_schema, schema_resolver, rw_columns, + error_ctx, }) } @@ -435,7 +440,8 @@ mod tests { .map(|c| SourceColumnDesc::from(&c)) .collect_vec(); - let parser = DebeziumAvroParser::new(columns.clone(), config)?; + let parser = + DebeziumAvroParser::new(columns.clone(), config, SourceErrorContext::for_test())?; let [(op, row)]: [_; 1] = parse_one(parser, columns, DEBEZIUM_AVRO_DATA) .await .try_into() diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index d0891697e4096..a233203b0a7bd 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -23,7 +23,7 @@ use super::operators::*; use crate::impl_common_parser_logic; use crate::parser::common::simd_json_parse_value; use crate::parser::{SourceStreamChunkRowWriter, WriteGuard}; -use crate::source::SourceColumnDesc; +use crate::source::{SourceColumnDesc, SourceErrorContext}; const BEFORE: &str = "before"; const AFTER: &str = "after"; @@ -43,11 +43,15 @@ impl_common_parser_logic!(DebeziumJsonParser); #[derive(Debug)] pub struct DebeziumJsonParser { pub(crate) rw_columns: Vec, + error_ctx: SourceErrorContext, } impl DebeziumJsonParser { - pub fn new(rw_columns: Vec) -> Result { - Ok(Self { rw_columns }) + pub fn new(rw_columns: Vec, error_ctx: SourceErrorContext) -> Result { + Ok(Self { + rw_columns, + error_ctx, + }) } #[allow(clippy::unused_async)] @@ -229,7 +233,8 @@ mod tests { let columns = get_test_columns(); - let parser = DebeziumJsonParser::new(columns.clone()).unwrap(); + let parser = + DebeziumJsonParser::new(columns.clone(), SourceErrorContext::for_test()).unwrap(); let [(_op, row)]: [_; 1] = parse_one(parser, columns, data).await.try_into().unwrap(); @@ -251,7 +256,8 @@ mod tests { let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"source":{"version":"1.7.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1639551564000,"snapshot":"false","db":"inventory","sequence":null,"table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1639551564960,"transaction":null}}"#; let columns = get_test_columns(); - let parser = DebeziumJsonParser::new(columns.clone()).unwrap(); + let parser = + DebeziumJsonParser::new(columns.clone(), SourceErrorContext::for_test()).unwrap(); let [(op, row)]: [_; 1] = parse_one(parser, columns, data).await.try_into().unwrap(); assert_eq!(op, Op::Insert); @@ -273,7 +279,8 @@ mod tests { let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":1.234},"after":null,"source":{"version":"1.7.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1639551767000,"snapshot":"false","db":"inventory","sequence":null,"table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1045,"row":0,"thread":null,"query":null},"op":"d","ts_ms":1639551767775,"transaction":null}}"#; let columns = get_test_columns(); - let parser = DebeziumJsonParser::new(columns.clone()).unwrap(); + let parser = + DebeziumJsonParser::new(columns.clone(), SourceErrorContext::for_test()).unwrap(); let [(op, row)]: [_; 1] = parse_one(parser, columns, data).await.try_into().unwrap(); assert_eq!(op, Op::Delete); @@ -302,7 +309,8 @@ mod tests { let columns = get_test_columns(); - let parser = DebeziumJsonParser::new(columns.clone()).unwrap(); + let parser = + DebeziumJsonParser::new(columns.clone(), SourceErrorContext::for_test()).unwrap(); let [(op1, row1), (op2, row2)]: [_; 2] = parse_one(parser, columns, data).await.try_into().unwrap(); @@ -333,7 +341,8 @@ mod tests { let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"source":{"version":"1.7.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1639551564000,"snapshot":"false","db":"inventory","sequence":null,"table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":null,"query":null},"op":"u","ts_ms":1639551564960,"transaction":null}}"#; let columns = get_test_columns(); - let parser = DebeziumJsonParser::new(columns.clone()).unwrap(); + let parser = + DebeziumJsonParser::new(columns.clone(), SourceErrorContext::for_test()).unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 2); let writer = builder.row_writer(); diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 208fad1986ffd..375c6422d9a70 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -18,11 +18,12 @@ use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; use simd_json::{BorrowedValue, ValueAccess}; +use crate::common::UpsertMessage; use crate::impl_common_parser_logic; use crate::parser::common::simd_json_parse_value; use crate::parser::util::at_least_one_ok; use crate::parser::{SourceStreamChunkRowWriter, WriteGuard}; -use crate::source::SourceColumnDesc; +use crate::source::{SourceColumnDesc, SourceErrorContext}; impl_common_parser_logic!(JsonParser); @@ -30,16 +31,41 @@ impl_common_parser_logic!(JsonParser); #[derive(Debug)] pub struct JsonParser { rw_columns: Vec, + error_ctx: SourceErrorContext, + enable_upsert: bool, } impl JsonParser { - pub fn new(rw_columns: Vec) -> Result { - Ok(Self { rw_columns }) + pub fn new(rw_columns: Vec, error_ctx: SourceErrorContext) -> Result { + Ok(Self { + rw_columns, + error_ctx, + enable_upsert: false, + }) + } + + pub fn new_for_test(rw_columns: Vec) -> Result { + Ok(Self { + rw_columns, + error_ctx: SourceErrorContext::for_test(), + enable_upsert: false, + }) + } + + pub fn new_with_upsert( + rw_columns: Vec, + error_ctx: SourceErrorContext, + ) -> Result { + Ok(Self { + rw_columns, + error_ctx, + enable_upsert: true, + }) } #[inline(always)] fn parse_single_value( - value: BorrowedValue<'_>, + value: &BorrowedValue<'_>, writer: &mut SourceStreamChunkRowWriter<'_>, ) -> Result { writer.insert(|desc| { @@ -60,27 +86,62 @@ impl JsonParser { payload: &[u8], mut writer: SourceStreamChunkRowWriter<'_>, ) -> Result { - let mut payload_mut = payload.to_vec(); + enum Op { + Insert, + Delete, + } + + let (_payload, op) = if self.enable_upsert { + let msg: UpsertMessage<'_> = bincode::deserialize(payload) + .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; + if !msg.record.is_empty() { + (msg.record, Op::Insert) + } else { + (msg.primary_key, Op::Delete) + } + } else { + (payload.into(), Op::Insert) + }; + + let mut payload_mut = _payload.to_vec(); let value: BorrowedValue<'_> = simd_json::to_borrowed_value(&mut payload_mut) .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; - let results = match value { - BorrowedValue::Array(objects) => objects - .into_iter() - .map(|obj| Self::parse_single_value(obj, &mut writer)) - .collect_vec(), - _ => { - return Self::parse_single_value(value, &mut writer); + if let BorrowedValue::Array(ref objects) = value && matches!(op, Op::Insert) { + at_least_one_ok( + objects + .iter() + .map(|obj| Self::parse_single_value(obj, &mut writer)) + .collect_vec(), + ) + } else { + let fill_fn = |desc: &SourceColumnDesc| { + simd_json_parse_value( + &desc.data_type, + value.get(desc.name.to_ascii_lowercase().as_str()), + ) + .map_err(|e| { + tracing::error!( + "failed to process value ({}): {}", + String::from_utf8_lossy(payload), + e + ); + e.into() + }) + }; + match op { + Op::Insert => writer.insert(fill_fn), + Op::Delete => writer.delete(fill_fn), } - }; - at_least_one_ok(results) + } } } #[cfg(test)] mod tests { use std::str::FromStr; + use std::vec; use itertools::Itertools; use risingwave_common::array::{Op, StructValue}; @@ -90,7 +151,9 @@ mod tests { use risingwave_common::types::{DataType, Decimal, ScalarImpl, ToOwnedDatum}; use risingwave_expr::vector_op::cast::{str_to_date, str_to_timestamp}; + use crate::common::UpsertMessage; use crate::parser::{JsonParser, SourceColumnDesc, SourceStreamChunkBuilder}; + use crate::source::SourceErrorContext; fn get_payload() -> Vec<&'static [u8]> { vec![ @@ -119,7 +182,7 @@ mod tests { SourceColumnDesc::simple("decimal", DataType::Decimal, 10.into()), ]; - let parser = JsonParser::new(descs.clone()).unwrap(); + let parser = JsonParser::new(descs.clone(), SourceErrorContext::for_test()).unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 2); @@ -218,7 +281,7 @@ mod tests { SourceColumnDesc::simple("v2", DataType::Int16, 1.into()), SourceColumnDesc::simple("v3", DataType::Varchar, 2.into()), ]; - let parser = JsonParser::new(descs.clone()).unwrap(); + let parser = JsonParser::new(descs.clone(), SourceErrorContext::for_test()).unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 3); // Parse a correct record. @@ -279,7 +342,7 @@ mod tests { .map(SourceColumnDesc::from) .collect_vec(); - let parser = JsonParser::new(descs.clone()).unwrap(); + let parser = JsonParser::new(descs.clone(), SourceErrorContext::for_test()).unwrap(); let payload = br#" { "data": { @@ -326,4 +389,69 @@ mod tests { ]; assert_eq!(row, expected); } + #[tokio::test] + async fn test_json_upsert_parser() { + let items = &[ + (r#"{"a":1}"#, r#"{"a":1,"b":2}"#), + (r#"{"a":1}"#, r#"{"a":1,"b":3}"#), + (r#"{"a":2}"#, r#"{"a":2,"b":2}"#), + (r#"{"a":2}"#, r#""#), + ] + .map(|(k, v)| { + bincode::serialize(&UpsertMessage { + primary_key: k.as_bytes().into(), + record: v.as_bytes().into(), + }) + .unwrap() + }); + let descs = vec![ + SourceColumnDesc::simple("a", DataType::Int32, 0.into()), + SourceColumnDesc::simple("b", DataType::Int32, 1.into()), + ]; + let parser = + JsonParser::new_with_upsert(descs.clone(), SourceErrorContext::for_test()).unwrap(); + let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 4); + for item in items { + parser + .parse_inner(item, builder.row_writer()) + .await + .unwrap(); + } + let chunk = builder.finish(); + let mut rows = chunk.rows(); + + { + let (op, row) = rows.next().unwrap(); + assert_eq!(op, Op::Insert); + assert_eq!( + row.datum_at(0).to_owned_datum(), + (Some(ScalarImpl::Int32(1))) + ); + } + + { + let (op, row) = rows.next().unwrap(); + assert_eq!(op, Op::Insert); + assert_eq!( + row.datum_at(0).to_owned_datum(), + (Some(ScalarImpl::Int32(1))) + ); + } + { + let (op, row) = rows.next().unwrap(); + assert_eq!(op, Op::Insert); + assert_eq!( + row.datum_at(0).to_owned_datum(), + (Some(ScalarImpl::Int32(2))) + ); + } + { + let (op, row) = rows.next().unwrap(); + assert_eq!(op, Op::Delete); + assert_eq!( + row.datum_at(0).to_owned_datum(), + (Some(ScalarImpl::Int32(2))) + ); + } + } } diff --git a/src/connector/src/parser/maxwell/simd_json_parser.rs b/src/connector/src/parser/maxwell/simd_json_parser.rs index 712b52747ab9e..17a9f9e70b165 100644 --- a/src/connector/src/parser/maxwell/simd_json_parser.rs +++ b/src/connector/src/parser/maxwell/simd_json_parser.rs @@ -23,7 +23,7 @@ use super::operators::*; use crate::impl_common_parser_logic; use crate::parser::common::simd_json_parse_value; use crate::parser::{SourceStreamChunkRowWriter, WriteGuard}; -use crate::source::SourceColumnDesc; +use crate::source::{SourceColumnDesc, SourceErrorContext}; const AFTER: &str = "data"; const BEFORE: &str = "old"; @@ -34,11 +34,15 @@ impl_common_parser_logic!(MaxwellParser); #[derive(Debug)] pub struct MaxwellParser { pub(crate) rw_columns: Vec, + error_ctx: SourceErrorContext, } impl MaxwellParser { - pub fn new(rw_columns: Vec) -> Result { - Ok(Self { rw_columns }) + pub fn new(rw_columns: Vec, error_ctx: SourceErrorContext) -> Result { + Ok(Self { + rw_columns, + error_ctx, + }) } #[allow(clippy::unused_async)] @@ -134,7 +138,7 @@ mod tests { SourceColumnDesc::simple("birthday", DataType::Timestamp, 3.into()), ]; - let parser = MaxwellParser::new(descs.clone()).unwrap(); + let parser = MaxwellParser::new(descs.clone(), SourceErrorContext::for_test()).unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 4); let payloads = vec![ diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index e75b49c1f8c0c..cf890760dcbe0 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -32,7 +32,8 @@ use risingwave_pb::catalog::StreamSourceInfo; pub use self::csv_parser::CsvParserConfig; use crate::parser::maxwell::MaxwellParser; use crate::source::{ - BoxSourceStream, BoxSourceWithStateStream, SourceColumnDesc, SourceFormat, StreamChunkWithState, + BoxSourceStream, BoxSourceWithStateStream, SourceColumnDesc, SourceErrorContext, SourceFormat, + StreamChunkWithState, }; mod avro; @@ -46,7 +47,6 @@ mod maxwell; mod protobuf; mod schema_registry; mod util; - /// A builder for building a [`StreamChunk`] from [`SourceColumnDesc`]. pub struct SourceStreamChunkBuilder { descs: Vec, @@ -244,11 +244,11 @@ impl SourceStreamChunkRowWriter<'_> { /// For other op like 'insert', 'update', 'delete', we will leave the hollow for the meta column /// builder. e.g after insert - /// `data_budiler` = [1], `meta_column_builder` = [], `op` = [insert] + /// `data_builder` = [1], `meta_column_builder` = [], `op` = [insert] /// /// This function is used to fulfill this hollow in `meta_column_builder`. /// e.g after fulfill - /// `data_budiler` = [1], `meta_column_builder` = [1], `op` = [insert] + /// `data_builder` = [1], `meta_column_builder` = [1], `op` = [insert] pub fn fulfill_meta_column( &mut self, mut f: impl FnMut(&SourceColumnDesc) -> Option, @@ -334,26 +334,33 @@ impl ByteStreamSourceParserImpl { } } - pub fn create(parser_config: ParserConfig) -> Result { + pub fn create(parser_config: ParserConfig, error_ctx: SourceErrorContext) -> Result { let CommonParserConfig { rw_columns } = parser_config.common; match parser_config.specific { - SpecificParserConfig::Csv(config) => CsvParser::new(rw_columns, config).map(Self::Csv), - SpecificParserConfig::Avro(config) => { - AvroParser::new(rw_columns, config).map(Self::Avro) + SpecificParserConfig::Csv(config) => { + CsvParser::new(rw_columns, config, error_ctx).map(Self::Csv) + } + SpecificParserConfig::Avro(config) | SpecificParserConfig::UpsertAvro(config) => { + AvroParser::new(rw_columns, config, error_ctx).map(Self::Avro) } SpecificParserConfig::Protobuf(config) => { - ProtobufParser::new(rw_columns, config).map(Self::Protobuf) + ProtobufParser::new(rw_columns, config, error_ctx).map(Self::Protobuf) + } + SpecificParserConfig::Json => JsonParser::new(rw_columns, error_ctx).map(Self::Json), + SpecificParserConfig::UpsertJson => { + JsonParser::new_with_upsert(rw_columns, error_ctx).map(Self::Json) } - SpecificParserConfig::Json => JsonParser::new(rw_columns).map(Self::Json), SpecificParserConfig::CanalJson => { - CanalJsonParser::new(rw_columns).map(Self::CanalJson) + CanalJsonParser::new(rw_columns, error_ctx).map(Self::CanalJson) } SpecificParserConfig::DebeziumJson => { - DebeziumJsonParser::new(rw_columns).map(Self::DebeziumJson) + DebeziumJsonParser::new(rw_columns, error_ctx).map(Self::DebeziumJson) + } + SpecificParserConfig::Maxwell => { + MaxwellParser::new(rw_columns, error_ctx).map(Self::Maxwell) } - SpecificParserConfig::Maxwell => MaxwellParser::new(rw_columns).map(Self::Maxwell), SpecificParserConfig::DebeziumAvro(config) => { - DebeziumAvroParser::new(rw_columns, config).map(Self::DebeziumAvro) + DebeziumAvroParser::new(rw_columns, config, error_ctx).map(Self::DebeziumAvro) } SpecificParserConfig::Native => { unreachable!("Native parser should not be created") @@ -377,8 +384,10 @@ pub struct CommonParserConfig { pub enum SpecificParserConfig { Csv(CsvParserConfig), Avro(AvroParserConfig), + UpsertAvro(AvroParserConfig), Protobuf(ProtobufParserConfig), Json, + UpsertJson, DebeziumJson, Maxwell, CanalJson, @@ -391,9 +400,11 @@ impl SpecificParserConfig { pub fn get_source_format(&self) -> SourceFormat { match self { SpecificParserConfig::Avro(_) => SourceFormat::Avro, + SpecificParserConfig::UpsertAvro(_) => SourceFormat::UpsertAvro, SpecificParserConfig::Csv(_) => SourceFormat::Csv, SpecificParserConfig::Protobuf(_) => SourceFormat::Protobuf, SpecificParserConfig::Json => SourceFormat::Json, + SpecificParserConfig::UpsertJson => SourceFormat::UpsertJson, SpecificParserConfig::DebeziumJson => SourceFormat::DebeziumJson, SpecificParserConfig::Maxwell => SourceFormat::Maxwell, SpecificParserConfig::CanalJson => SourceFormat::CanalJson, @@ -413,8 +424,28 @@ impl SpecificParserConfig { has_header: info.csv_has_header, }), SourceFormat::Avro => SpecificParserConfig::Avro( - AvroParserConfig::new(props, &info.row_schema_location, info.use_schema_registry) - .await?, + AvroParserConfig::new( + props, + &info.row_schema_location, + info.use_schema_registry, + false, + None, + ) + .await?, + ), + SourceFormat::UpsertAvro => SpecificParserConfig::UpsertAvro( + AvroParserConfig::new( + props, + &info.row_schema_location, + info.use_schema_registry, + true, + if info.upsert_avro_primary_key.is_empty() { + None + } else { + Some(info.upsert_avro_primary_key.to_string()) + }, + ) + .await?, ), SourceFormat::Protobuf => SpecificParserConfig::Protobuf( ProtobufParserConfig::new( @@ -426,6 +457,7 @@ impl SpecificParserConfig { .await?, ), SourceFormat::Json => SpecificParserConfig::Json, + SourceFormat::UpsertJson => SpecificParserConfig::UpsertJson, SourceFormat::DebeziumJson => SpecificParserConfig::DebeziumJson, SourceFormat::Maxwell => SpecificParserConfig::Maxwell, SourceFormat::CanalJson => SpecificParserConfig::CanalJson, diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index bd4e9f69a483d..7ab89ce6de679 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -33,7 +33,7 @@ use crate::impl_common_parser_logic; use crate::parser::schema_registry::{extract_schema_id, Client}; use crate::parser::util::get_kafka_topic; use crate::parser::{SourceStreamChunkRowWriter, WriteGuard}; -use crate::source::SourceColumnDesc; +use crate::source::{SourceColumnDesc, SourceErrorContext}; impl_common_parser_logic!(ProtobufParser); @@ -42,6 +42,7 @@ pub struct ProtobufParser { message_descriptor: MessageDescriptor, confluent_wire_type: bool, rw_columns: Vec, + error_ctx: SourceErrorContext, } #[derive(Debug, Clone)] @@ -167,7 +168,11 @@ impl ProtobufParserConfig { } impl ProtobufParser { - pub fn new(rw_columns: Vec, config: ProtobufParserConfig) -> Result { + pub fn new( + rw_columns: Vec, + config: ProtobufParserConfig, + error_ctx: SourceErrorContext, + ) -> Result { let ProtobufParserConfig { confluent_wire_type, message_descriptor, @@ -176,6 +181,7 @@ impl ProtobufParser { message_descriptor, confluent_wire_type, rw_columns, + error_ctx, }) } @@ -362,7 +368,7 @@ mod test { println!("location: {}", location); let conf = ProtobufParserConfig::new(&HashMap::new(), &location, message_name, false).await?; - let parser = ProtobufParser::new(Vec::default(), conf)?; + let parser = ProtobufParser::new(Vec::default(), conf, SourceErrorContext::for_test())?; let value = DynamicMessage::decode(parser.message_descriptor, PRE_GEN_PROTO_DATA).unwrap(); assert_eq!( diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index ea9e659938d8f..4f4d1376c9099 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -80,13 +80,60 @@ pub trait SplitEnumerator: Sized { pub struct SourceInfo { pub actor_id: u32, pub source_id: TableId, + // There should be a 1-1 mapping between `source_id` & `fragment_id` + pub fragment_id: u32, } impl SourceInfo { - pub fn new(actor_id: u32, source_id: TableId) -> Self { + pub fn new(actor_id: u32, source_id: TableId, fragment_id: u32) -> Self { SourceInfo { actor_id, source_id, + fragment_id, + } + } +} + +#[derive(Debug, Clone)] +pub struct SourceErrorContext { + metrics: Arc, + table_id: u32, + fragment_id: u32, +} + +impl SourceErrorContext { + pub(crate) fn new(table_id: u32, fragment_id: u32, metrics: Arc) -> Self { + Self { + metrics, + table_id, + fragment_id, + } + } + + pub(crate) fn report_stream_source_error(&self, e: &RwError) { + // Do not report for batch + if self.fragment_id == u32::MAX { + return; + } + self.metrics + .user_source_error_count + .with_label_values(&[ + "SourceError", + // TODO(jon-chuang): add the error msg truncator to truncate these + &e.inner().to_string(), + // Let's be a bit more specific for SourceExecutor + "SourceExecutor", + &self.fragment_id.to_string(), + &self.table_id.to_string(), + ]) + .inc(); + } + + pub(crate) fn for_test() -> Self { + Self { + metrics: Arc::new(SourceMetrics::unused()), + table_id: u32::MAX, + fragment_id: u32::MAX, } } } @@ -95,9 +142,11 @@ impl SourceInfo { pub enum SourceFormat { Invalid, Json, + UpsertJson, Protobuf, DebeziumJson, Avro, + UpsertAvro, Maxwell, CanalJson, Csv, diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index b14eca23708ce..ccbd3ff4ab821 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -34,8 +34,8 @@ use crate::source::filesystem::file_common::FsSplit; use crate::source::filesystem::s3::S3Properties; use crate::source::monitor::SourceMetrics; use crate::source::{ - BoxSourceWithStateStream, Column, SourceInfo, SourceMessage, SourceMeta, SplitImpl, - StreamChunkWithState, + BoxSourceWithStateStream, Column, SourceErrorContext, SourceInfo, SourceMessage, SourceMeta, + SplitImpl, StreamChunkWithState, }; const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; const STREAM_READER_CAPACITY: usize = 4096; @@ -189,6 +189,12 @@ impl S3FileReader { for split in self.splits { let actor_id = self.source_info.actor_id.to_string(); let source_id = self.source_info.source_id.to_string(); + let error_ctx = SourceErrorContext::new( + self.source_info.source_id.table_id, + self.source_info.fragment_id, + self.metrics.clone(), + ); + let split_id = split.id(); let data_stream = Self::stream_read_object( @@ -199,7 +205,7 @@ impl S3FileReader { self.source_info, ); - let parser = ByteStreamSourceParserImpl::create(self.parser_config.clone())?; + let parser = ByteStreamSourceParserImpl::create(self.parser_config.clone(), error_ctx)?; let msg_stream = parser.into_stream(Box::pin(data_stream)); #[for_await] for msg in msg_stream { diff --git a/src/connector/src/source/kafka/mod.rs b/src/connector/src/source/kafka/mod.rs index 24e350b1a0783..f7dd3ae9cea5b 100644 --- a/src/connector/src/source/kafka/mod.rs +++ b/src/connector/src/source/kafka/mod.rs @@ -53,6 +53,12 @@ pub struct KafkaProperties { #[serde(rename = "properties.group.id", alias = "kafka.consumer.group")] pub consumer_group: Option, + /// This parameter is used to tell KafkaSplitReader to produce `UpsertMessage`s, which + /// combine both key and value fields of the Kafka message. + /// TODO: Currently, `Option` can not be parsed here. + #[serde(rename = "upsert")] + pub upsert: Option, + #[serde(flatten)] pub common: KafkaCommon, } diff --git a/src/connector/src/source/kafka/source/message.rs b/src/connector/src/source/kafka/source/message.rs index 6c52455c779e7..6c56add580d3e 100644 --- a/src/connector/src/source/kafka/source/message.rs +++ b/src/connector/src/source/kafka/source/message.rs @@ -16,6 +16,7 @@ use bytes::Bytes; use rdkafka::message::BorrowedMessage; use rdkafka::Message; +use crate::common::UpsertMessage; use crate::source::base::SourceMessage; use crate::source::SourceMeta; @@ -25,6 +26,25 @@ pub struct KafkaMeta { pub timestamp: Option, } +impl SourceMessage { + pub fn from_kafka_message_upsert(message: BorrowedMessage<'_>) -> Self { + let encoded = bincode::serialize(&UpsertMessage { + primary_key: message.key().unwrap_or_default().into(), + record: message.payload().unwrap_or_default().into(), + }) + .unwrap(); + SourceMessage { + // TODO(TaoWu): Possible performance improvement: avoid memory copying here. + payload: Some(encoded.into()), + offset: message.offset().to_string(), + split_id: message.partition().to_string().into(), + meta: SourceMeta::Kafka(KafkaMeta { + timestamp: message.timestamp().to_millis(), + }), + } + } +} + impl<'a> From> for SourceMessage { fn from(message: BorrowedMessage<'a>) -> Self { SourceMessage { diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index 2ad056953327f..e171a6cc39b0f 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -41,6 +41,7 @@ pub struct KafkaSplitReader { stop_offset: Option, bytes_per_second: usize, max_num_messages: usize, + enable_upsert: bool, split_id: SplitId, parser_config: ParserConfig, @@ -139,6 +140,11 @@ impl SplitReader for KafkaSplitReader { parser_config, metrics, source_info, + enable_upsert: properties + .upsert + .as_ref() + .filter(|x| *x == "true") + .is_some(), }) } @@ -174,7 +180,12 @@ impl KafkaSplitReader { Some(payload) => payload.len(), }; num_messages += 1; - res.push(SourceMessage::from(msg)); + if self.enable_upsert { + res.push(SourceMessage::from_kafka_message_upsert(msg)); + } else { + res.push(SourceMessage::from(msg)); + } + if let Some(stop_offset) = self.stop_offset { if cur_offset == stop_offset - 1 { tracing::debug!( diff --git a/src/connector/src/source/monitor/metrics.rs b/src/connector/src/source/monitor/metrics.rs index c0e922bb734df..89f753e163cda 100644 --- a/src/connector/src/source/monitor/metrics.rs +++ b/src/connector/src/source/monitor/metrics.rs @@ -20,6 +20,8 @@ pub struct SourceMetrics { pub registry: Registry, pub partition_input_count: GenericCounterVec, pub partition_input_bytes: GenericCounterVec, + /// User error reporting + pub user_source_error_count: GenericCounterVec, } impl SourceMetrics { @@ -38,10 +40,24 @@ impl SourceMetrics { registry ) .unwrap(); + let user_source_error_count = register_int_counter_vec_with_registry!( + "user_source_error_count", + "Source errors in the system, queryable by tags", + &[ + "error_type", + "error_msg", + "executor_name", + "fragment_id", + "table_id" + ], + registry, + ) + .unwrap(); SourceMetrics { registry, partition_input_count, partition_input_bytes, + user_source_error_count, } } diff --git a/src/expr/Cargo.toml b/src/expr/Cargo.toml index cc09f522ca9d8..bc2d71a93b1d1 100644 --- a/src/expr/Cargo.toml +++ b/src/expr/Cargo.toml @@ -17,8 +17,8 @@ normal = ["workspace-hack"] [dependencies] aho-corasick = "0.7" anyhow = "1" -arrow-array = "31" -arrow-schema = "31" +arrow-array = "33" +arrow-schema = "33" chrono = { version = "0.4", default-features = false, features = ["clock", "std"] } chrono-tz = { version = "0.7", features = ["case-insensitive"] } dyn-clone = "1" diff --git a/src/expr/src/expr/build_expr_from_prost.rs b/src/expr/src/expr/build_expr_from_prost.rs index b7f6ef27d337d..34064d2cdafe4 100644 --- a/src/expr/src/expr/build_expr_from_prost.rs +++ b/src/expr/src/expr/build_expr_from_prost.rs @@ -64,11 +64,15 @@ pub fn build_from_prost(prost: &ExprNode) -> Result { // Fixed number of arguments and based on `Unary/Binary/Ternary/...Expression` Cast | Upper | Lower | Md5 | Not | IsTrue | IsNotTrue | IsFalse | IsNotFalse | IsNull | IsNotNull | Neg | Ascii | Abs | Ceil | Floor | Round | Exp | BitwiseNot | CharLength - | BoolOut | OctetLength | BitLength | ToTimestamp => build_unary_expr_prost(prost), + | BoolOut | OctetLength | BitLength | ToTimestamp | JsonbTypeof | JsonbArrayLength => { + build_unary_expr_prost(prost) + } Equal | NotEqual | LessThan | LessThanOrEqual | GreaterThan | GreaterThanOrEqual | Add | Subtract | Multiply | Divide | Modulus | Extract | RoundDigit | Pow | TumbleStart | Position | BitwiseShiftLeft | BitwiseShiftRight | BitwiseAnd | BitwiseOr | BitwiseXor - | ConcatOp | AtTimeZone | CastWithTimeZone => build_binary_expr_prost(prost), + | ConcatOp | AtTimeZone | CastWithTimeZone | JsonbAccessInner | JsonbAccessStr => { + build_binary_expr_prost(prost) + } And | Or | IsDistinctFrom | IsNotDistinctFrom | ArrayAccess | FormatType => { build_nullable_binary_expr_prost(prost) } diff --git a/src/expr/src/expr/expr_array_to_string.rs b/src/expr/src/expr/expr_array_to_string.rs index 213e164d5ead6..d5064c64fd585 100644 --- a/src/expr/src/expr/expr_array_to_string.rs +++ b/src/expr/src/expr/expr_array_to_string.rs @@ -175,8 +175,12 @@ impl Expression for ArrayToStringExpression { let array = self.array.eval_row(input)?; let delimiter = self.delimiter.eval_row(input)?; - let result = if let Some(array) = array && let Some(delimiter) = delimiter && let Some(e) = &self.null_string { - let null_string = e.eval_row(input)?; + let result = if let Some(array) = array && let Some(delimiter) = delimiter { + let null_string = if let Some(e) = &self.null_string { + e.eval_row(input)? + } else { + None + }; let mut writer = String::new(); if let Some(null_string) = null_string { self.evaluate_with_nulls(array.as_scalar_ref_impl().into_list(), delimiter.as_utf8(), null_string.as_utf8(), &mut writer); diff --git a/src/expr/src/expr/expr_binary_nonnull.rs b/src/expr/src/expr/expr_binary_nonnull.rs index 910b563af67f8..7d1e7488153c8 100644 --- a/src/expr/src/expr/expr_binary_nonnull.rs +++ b/src/expr/src/expr/expr_binary_nonnull.rs @@ -13,14 +13,17 @@ // limitations under the License. use risingwave_common::array::{ - Array, BoolArray, DecimalArray, F64Array, I32Array, I64Array, IntervalArray, ListArray, - NaiveDateArray, NaiveDateTimeArray, StructArray, Utf8Array, + Array, BoolArray, DecimalArray, F64Array, I32Array, I64Array, IntervalArray, JsonbArrayBuilder, + ListArray, NaiveDateArray, NaiveDateTimeArray, StructArray, Utf8Array, Utf8ArrayBuilder, }; use risingwave_common::types::*; use risingwave_pb::expr::expr_node::Type; use super::Expression; use crate::expr::expr_binary_bytes::new_concat_op; +use crate::expr::expr_jsonb_access::{ + jsonb_array_element, jsonb_object_field, JsonbAccessExpression, +}; use crate::expr::template::{BinaryBytesExpression, BinaryExpression}; use crate::expr::{template_fast, BoxedExpression}; use crate::vector_op::arithmetic_op::*; @@ -680,6 +683,38 @@ pub fn new_binary_expr( )), Type::TumbleStart => new_tumble_start(l, r, ret)?, Type::ConcatOp => new_concat_op(l, r, ret), + Type::JsonbAccessInner => match r.return_type() { + DataType::Varchar => { + JsonbAccessExpression::::new_expr( + l, + r, + jsonb_object_field, + ) + .boxed() + } + DataType::Int32 => JsonbAccessExpression::::new_expr( + l, + r, + jsonb_array_element, + ) + .boxed(), + t => return Err(ExprError::UnsupportedFunction(format!("jsonb -> {t}"))), + }, + Type::JsonbAccessStr => match r.return_type() { + DataType::Varchar => JsonbAccessExpression::::new_expr( + l, + r, + jsonb_object_field, + ) + .boxed(), + DataType::Int32 => JsonbAccessExpression::::new_expr( + l, + r, + jsonb_array_element, + ) + .boxed(), + t => return Err(ExprError::UnsupportedFunction(format!("jsonb ->> {t}"))), + }, tp => { return Err(ExprError::UnsupportedFunction(format!( diff --git a/src/expr/src/expr/expr_jsonb_access.rs b/src/expr/src/expr/expr_jsonb_access.rs new file mode 100644 index 0000000000000..7c26779a39109 --- /dev/null +++ b/src/expr/src/expr/expr_jsonb_access.rs @@ -0,0 +1,222 @@ +// 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 either::Either; +use risingwave_common::array::{ + Array, ArrayBuilder, ArrayImpl, ArrayRef, DataChunk, JsonbArray, JsonbArrayBuilder, JsonbRef, + Utf8ArrayBuilder, +}; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, Datum, Scalar, ScalarRef}; +use risingwave_common::util::iter_util::ZipEqFast; + +use super::{BoxedExpression, Expression}; + +/// This is forked from [`BinaryExpression`] for the following reasons: +/// * Optimize for the case when rhs path is const. (not implemented yet) +/// * It can return null when neither input is null. +/// * We could `append(RefItem)` directly rather than getting a `OwnedItem` first. +pub struct JsonbAccessExpression { + input: BoxedExpression, + path: Either, + func: F, + _phantom: std::marker::PhantomData, +} + +impl std::fmt::Debug for JsonbAccessExpression { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("JsonbAccessExpression") + .field("input", &self.input) + .field("path", &self.path) + .finish() + } +} + +impl JsonbAccessExpression +where + F: Send + Sync + for<'a> Fn(JsonbRef<'a>, A::RefItem<'_>) -> Option>, +{ + #[expect(dead_code)] + pub fn new_const(input: BoxedExpression, path: A::OwnedItem, func: F) -> Self { + Self { + input, + path: Either::Right(path), + func, + _phantom: std::marker::PhantomData, + } + } + + pub fn new_expr(input: BoxedExpression, path: BoxedExpression, func: F) -> Self { + Self { + input, + path: Either::Left(path), + func, + _phantom: std::marker::PhantomData, + } + } + + pub fn eval_strict<'a>( + &self, + v: Option>, + p: Option>, + ) -> Option> { + match (v, p) { + (Some(v), Some(p)) => (self.func)(v, p), + _ => None, + } + } +} + +impl Expression for JsonbAccessExpression +where + A: Array, + for<'a> &'a A: From<&'a ArrayImpl>, + O: AccessOutput, + F: Send + Sync + for<'a> Fn(JsonbRef<'a>, A::RefItem<'_>) -> Option>, +{ + fn return_type(&self) -> DataType { + O::return_type() + } + + fn eval(&self, input: &DataChunk) -> crate::Result { + let Either::Left(path_expr) = &self.path else { + unreachable!("optimization for const path not implemented yet"); + }; + let path_array = path_expr.eval_checked(input)?; + let path_array: &A = path_array.as_ref().into(); + + let input_array = self.input.eval_checked(input)?; + let input_array: &JsonbArray = input_array.as_ref().into(); + + let mut builder = O::new(input.capacity()); + match input.visibility() { + // We could ignore visibility and always evaluate access path for all values, because it + // never returns runtime error. But using visibility could save us some clone cost, + // unless we adjust [`JsonbArray`] to make sure all clones are on [`Arc`]. + Some(visibility) => { + for ((v, p), visible) in input_array + .iter() + .zip_eq_fast(path_array.iter()) + .zip_eq_fast(visibility.iter()) + { + let r = visible.then(|| self.eval_strict(v, p)).flatten(); + builder.output_nullable(r)?; + } + } + None => { + for (v, p) in input_array.iter().zip_eq_fast(path_array.iter()) { + builder.output_nullable(self.eval_strict(v, p))?; + } + } + }; + Ok(std::sync::Arc::new(builder.finish().into())) + } + + fn eval_row(&self, input: &OwnedRow) -> crate::Result { + let Either::Left(path_expr) = &self.path else { + unreachable!("optimization for const path not implemented yet"); + }; + let p = path_expr.eval_row(input)?; + let p = p + .as_ref() + .map(|p| p.as_scalar_ref_impl().try_into().unwrap()); + + let v = self.input.eval_row(input)?; + let v = v + .as_ref() + .map(|v| v.as_scalar_ref_impl().try_into().unwrap()); + + let r = self.eval_strict(v, p); + Ok(r.and_then(O::to_datum)) + } +} + +pub fn jsonb_object_field<'a>(v: JsonbRef<'a>, p: &str) -> Option> { + v.access_object_field(p) +} + +pub fn jsonb_array_element(v: JsonbRef<'_>, p: i32) -> Option> { + let idx = if p < 0 { + let Ok(len) = v.array_len() else { + return None; + }; + if ((-p) as usize) > len { + return None; + } else { + len - ((-p) as usize) + } + } else { + p as usize + }; + v.access_array_element(idx) +} + +trait AccessOutput: ArrayBuilder { + fn return_type() -> DataType; + fn output(&mut self, v: JsonbRef<'_>) -> crate::Result<()>; + fn to_datum(v: JsonbRef<'_>) -> Datum; + fn output_nullable(&mut self, v: Option>) -> crate::Result<()> { + match v { + Some(v) => self.output(v)?, + None => self.append_null(), + }; + Ok(()) + } +} + +impl AccessOutput for JsonbArrayBuilder { + fn return_type() -> DataType { + DataType::Jsonb + } + + fn output(&mut self, v: JsonbRef<'_>) -> crate::Result<()> { + self.append(Some(v)); + Ok(()) + } + + fn to_datum(v: JsonbRef<'_>) -> Datum { + Some(v.to_owned_scalar().to_scalar_value()) + } +} + +impl AccessOutput for Utf8ArrayBuilder { + fn return_type() -> DataType { + DataType::Varchar + } + + fn output(&mut self, v: JsonbRef<'_>) -> crate::Result<()> { + match v.is_jsonb_null() { + true => self.append_null(), + false => { + let mut writer = self.writer().begin(); + v.force_str(&mut writer) + .map_err(|e| crate::ExprError::Internal(e.into()))?; + writer.finish(); + } + }; + Ok(()) + } + + fn to_datum(v: JsonbRef<'_>) -> Datum { + match v.is_jsonb_null() { + true => None, + false => { + let mut s = String::new(); + v.force_str(&mut s).unwrap(); + let s: Box = s.into(); + Some(s.to_scalar_value()) + } + } + } +} diff --git a/src/expr/src/expr/expr_udf.rs b/src/expr/src/expr/expr_udf.rs index b64f01add35c0..2726a0b75376a 100644 --- a/src/expr/src/expr/expr_udf.rs +++ b/src/expr/src/expr/expr_udf.rs @@ -15,7 +15,7 @@ use std::convert::TryFrom; use std::sync::Arc; -use arrow_schema::{Field, Schema}; +use arrow_schema::{Field, Schema, SchemaRef}; use risingwave_common::array::{ArrayImpl, ArrayRef, DataChunk}; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, Datum}; @@ -33,6 +33,7 @@ pub struct UdfExpression { // name: String, arg_types: Vec, return_type: DataType, + arg_schema: SchemaRef, client: ArrowFlightUdfClient, function_id: FunctionId, } @@ -51,10 +52,13 @@ impl Expression for UdfExpression { let columns: Vec<_> = self .children .iter() - .map(|c| c.eval_checked(input).map(|a| ("", a.as_ref().into()))) + .map(|c| c.eval_checked(input).map(|a| a.as_ref().into())) .try_collect()?; + let opts = + arrow_array::RecordBatchOptions::default().with_row_count(Some(input.cardinality())); let input = - arrow_array::RecordBatch::try_from_iter(columns).expect("failed to build record batch"); + arrow_array::RecordBatch::try_new_with_options(self.arg_schema.clone(), columns, &opts) + .expect("failed to build record batch"); let output = tokio::task::block_in_place(|| { tokio::runtime::Handle::current().block_on(self.client.call(&self.function_id, input)) })?; @@ -87,18 +91,18 @@ impl<'a> TryFrom<&'a ExprNode> for UdfExpression { bail!("expect UDF"); }; // connect to UDF service and check the function - let (client, function_id) = tokio::task::block_in_place(|| { + let (client, function_id, arg_schema) = tokio::task::block_in_place(|| { tokio::runtime::Handle::current().block_on(async { let client = ArrowFlightUdfClient::connect(&udf.path).await?; - let args = Schema::new( + let args = Arc::new(Schema::new( udf.arg_types .iter() .map(|t| Field::new("", DataType::from(t).into(), true)) .collect(), - ); + )); let returns = Schema::new(vec![Field::new("", (&return_type).into(), true)]); let id = client.check(&udf.name, &args, &returns).await?; - Ok((client, id)) as risingwave_udf::Result<_> + Ok((client, id, args)) as risingwave_udf::Result<_> }) })?; Ok(Self { @@ -106,6 +110,7 @@ impl<'a> TryFrom<&'a ExprNode> for UdfExpression { // name: udf.name.clone(), arg_types: udf.arg_types.iter().map(|t| t.into()).collect(), return_type, + arg_schema, client, function_id, }) diff --git a/src/expr/src/expr/expr_unary.rs b/src/expr/src/expr/expr_unary.rs index 00f9151078a0c..b957fb74d9628 100644 --- a/src/expr/src/expr/expr_unary.rs +++ b/src/expr/src/expr/expr_unary.rs @@ -30,6 +30,7 @@ use crate::vector_op::cast::*; use crate::vector_op::cmp::{is_false, is_not_false, is_not_true, is_true}; use crate::vector_op::conjunction; use crate::vector_op::exp::exp_f64; +use crate::vector_op::jsonb_info::{jsonb_array_length, jsonb_typeof}; use crate::vector_op::length::{bit_length, length_default, octet_length}; use crate::vector_op::lower::lower; use crate::vector_op::ltrim::ltrim; @@ -307,6 +308,18 @@ pub fn new_unary_expr( f64_sec_to_timestamptz, )) } + (ProstType::JsonbTypeof, DataType::Varchar, DataType::Jsonb) => { + UnaryBytesExpression::::new(child_expr, return_type, jsonb_typeof) + .boxed() + } + (ProstType::JsonbArrayLength, DataType::Int32, DataType::Jsonb) => { + UnaryExpression::::new( + child_expr, + return_type, + jsonb_array_length, + ) + .boxed() + } (expr, ret, child) => { return Err(ExprError::UnsupportedFunction(format!( "{:?}({:?}) -> {:?}", diff --git a/src/expr/src/expr/mod.rs b/src/expr/src/expr/mod.rs index 08fa3b06adb8f..8ff9e80a01033 100644 --- a/src/expr/src/expr/mod.rs +++ b/src/expr/src/expr/mod.rs @@ -44,6 +44,7 @@ mod expr_field; mod expr_in; mod expr_input_ref; mod expr_is_null; +mod expr_jsonb_access; mod expr_literal; mod expr_nested_construct; mod expr_quaternary_bytes; diff --git a/src/expr/src/sig/func.rs b/src/expr/src/sig/func.rs index 5af8eced4b0a7..1b160861ca666 100644 --- a/src/expr/src/sig/func.rs +++ b/src/expr/src/sig/func.rs @@ -309,6 +309,13 @@ fn build_type_derive_map() -> FuncSigMap { T::Varchar, ); + map.insert(E::JsonbAccessInner, vec![T::Jsonb, T::Int32], T::Jsonb); + map.insert(E::JsonbAccessInner, vec![T::Jsonb, T::Varchar], T::Jsonb); + map.insert(E::JsonbAccessStr, vec![T::Jsonb, T::Int32], T::Varchar); + map.insert(E::JsonbAccessStr, vec![T::Jsonb, T::Varchar], T::Varchar); + map.insert(E::JsonbTypeof, vec![T::Jsonb], T::Varchar); + map.insert(E::JsonbArrayLength, vec![T::Jsonb], T::Int32); + map } diff --git a/src/expr/src/vector_op/jsonb_info.rs b/src/expr/src/vector_op/jsonb_info.rs new file mode 100644 index 0000000000000..064e54959b3bd --- /dev/null +++ b/src/expr/src/vector_op/jsonb_info.rs @@ -0,0 +1,36 @@ +// 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::fmt::Write; + +use risingwave_common::array::JsonbRef; + +use crate::{ExprError, Result}; + +#[inline(always)] +pub fn jsonb_typeof(v: JsonbRef<'_>, writer: &mut dyn Write) -> Result<()> { + writer + .write_str(v.type_name()) + .map_err(|e| ExprError::Internal(e.into())) +} + +#[inline(always)] +pub fn jsonb_array_length(v: JsonbRef<'_>) -> Result { + v.array_len() + .map(|n| n as i32) + .map_err(|e| ExprError::InvalidParam { + name: "", + reason: e, + }) +} diff --git a/src/expr/src/vector_op/mod.rs b/src/expr/src/vector_op/mod.rs index d3f07ad7fed9d..164e433ad9dde 100644 --- a/src/expr/src/vector_op/mod.rs +++ b/src/expr/src/vector_op/mod.rs @@ -25,6 +25,7 @@ pub mod date_trunc; pub mod exp; pub mod extract; pub mod format_type; +pub mod jsonb_info; pub mod length; pub mod like; pub mod lower; diff --git a/src/frontend/planner_test/tests/testdata/agg.yaml b/src/frontend/planner_test/tests/testdata/agg.yaml index 756b80290e10c..6fe98048b213a 100644 --- a/src/frontend/planner_test/tests/testdata/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/agg.yaml @@ -30,7 +30,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, agg], pk_columns: [v1] } + StreamMaterialize { columns: [v1, agg], pk_columns: [v1], pk_conflict: "no check" } └─StreamProject { exprs: [t.v1, (min(t.v2) + (max(t.v3) * count(t.v1))) as $expr1] } └─StreamHashAgg { group_key: [t.v1], aggs: [count, min(t.v2), max(t.v3), count(t.v1)] } └─StreamExchange { dist: HashShard(t.v1) } @@ -50,7 +50,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [agg], pk_columns: [] } + StreamMaterialize { columns: [agg], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v2)) * sum0(count(t.v3)))) as $expr2] } └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), max(max(t.v2)), sum0(count(t.v3))] } └─StreamExchange { dist: Single } @@ -82,7 +82,7 @@ └─BatchProject { exprs: [t.v3, t.v1, (t.v1 + t.v2) as $expr1] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v3, agg], pk_columns: [v3] } + StreamMaterialize { columns: [v3, agg], pk_columns: [v3], pk_conflict: "no check" } └─StreamProject { exprs: [t.v3, (min(t.v1) * (sum($expr1)::Decimal / count($expr1))) as $expr2] } └─StreamHashAgg { group_key: [t.v3], aggs: [count, min(t.v1), sum($expr1), count($expr1)] } └─StreamExchange { dist: HashShard(t.v3) } @@ -150,7 +150,7 @@ └─BatchProject { exprs: [(t.v1 + t.v2) as $expr1] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [cnt, sum], pk_columns: [] } + StreamMaterialize { columns: [cnt, sum], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count($expr1)), sum(sum($expr1))] } └─StreamGlobalSimpleAgg { aggs: [count, sum0(count($expr1)), sum(sum($expr1))] } └─StreamExchange { dist: Single } @@ -168,7 +168,7 @@ └─BatchProject { exprs: [t.v1, (t.v2 + t.v3) as $expr1] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, agg], pk_columns: [v1] } + StreamMaterialize { columns: [v1, agg], pk_columns: [v1], pk_conflict: "no check" } └─StreamProject { exprs: [t.v1, ((sum($expr1) / count($expr1)) + max(t.v1)) as $expr2] } └─StreamHashAgg { group_key: [t.v1], aggs: [count, sum($expr1), count($expr1), max(t.v1)] } └─StreamExchange { dist: HashShard(t.v1) } @@ -420,7 +420,7 @@ └─BatchSimpleAgg { aggs: [min(t.v1), max(t.v3), count(t.v2)] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [agg], pk_columns: [] } + StreamMaterialize { columns: [agg], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v3)) * sum0(count(t.v2)))) as $expr2] } └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), max(max(t.v3)), sum0(count(t.v2))] } └─StreamExchange { dist: Single } @@ -442,7 +442,7 @@ └─LogicalAgg { group_key: [t.v1], aggs: [] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | - StreamMaterialize { columns: [v1, t.v1(hidden)], pk_columns: [t.v1] } + StreamMaterialize { columns: [v1, t.v1(hidden)], pk_columns: [t.v1], pk_conflict: "no check" } └─StreamProject { exprs: [t.v1, t.v1] } └─StreamHashAgg { group_key: [t.v1], aggs: [count] } └─StreamExchange { dist: HashShard(t.v1) } @@ -462,7 +462,7 @@ └─LogicalAgg { group_key: [t.v3, t.v2], aggs: [min(t.v1), max(t.v1)] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3] } stream_plan: | - StreamMaterialize { columns: [v2, min_v1, v3, max_v1, t.v2(hidden)], pk_columns: [v3, t.v2] } + StreamMaterialize { columns: [v2, min_v1, v3, max_v1, t.v2(hidden)], pk_columns: [v3, t.v2], pk_conflict: "no check" } └─StreamProject { exprs: [t.v2, min(t.v1), t.v3, max(t.v1), t.v2] } └─StreamHashAgg { group_key: [t.v3, t.v2], aggs: [count, min(t.v1), max(t.v1)] } └─StreamExchange { dist: HashShard(t.v2, t.v3) } @@ -480,7 +480,7 @@ LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | - StreamMaterialize { columns: [s1], pk_columns: [] } + StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1))] } └─StreamExchange { dist: Single } @@ -499,7 +499,7 @@ LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | - StreamMaterialize { columns: [s1], pk_columns: [] } + StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1))] } └─StreamExchange { dist: Single } @@ -518,7 +518,7 @@ LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | - StreamMaterialize { columns: [s1], pk_columns: [] } + StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1))] } └─StreamExchange { dist: Single } @@ -537,7 +537,7 @@ LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | - StreamMaterialize { columns: [s1], pk_columns: [] } + StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1))] } └─StreamExchange { dist: Single } @@ -556,7 +556,7 @@ LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | - StreamMaterialize { columns: [s1], pk_columns: [] } + StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1))] } └─StreamExchange { dist: Single } @@ -575,7 +575,7 @@ LogicalAgg { aggs: [sum(t.v1) filter((t.v1 > 0:Int32))] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | - StreamMaterialize { columns: [sa], pk_columns: [] } + StreamMaterialize { columns: [sa], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1) filter((t.v1 > 0:Int32)))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1) filter((t.v1 > 0:Int32)))] } └─StreamExchange { dist: Single } @@ -610,7 +610,7 @@ └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: | - StreamMaterialize { columns: [sab], pk_columns: [] } + StreamMaterialize { columns: [sab], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))))] } └─StreamExchange { dist: Single } @@ -632,7 +632,7 @@ └─LogicalAgg { group_key: [t.b], aggs: [sum(t.a) filter((t.a > t.b)), count(t.a) filter((t.a > t.b))] } └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: | - StreamMaterialize { columns: [avga, t.b(hidden)], pk_columns: [t.b] } + StreamMaterialize { columns: [avga, t.b(hidden)], pk_columns: [t.b], pk_conflict: "no check" } └─StreamProject { exprs: [(sum(t.a) filter((t.a > t.b))::Decimal / count(t.a) filter((t.a > t.b))) as $expr1, t.b] } └─StreamHashAgg { group_key: [t.b], aggs: [count, sum(t.a) filter((t.a > t.b)), count(t.a) filter((t.a > t.b))] } └─StreamExchange { dist: HashShard(t.b) } @@ -650,7 +650,7 @@ LogicalAgg { aggs: [count filter((t.a > t.b))] } └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: | - StreamMaterialize { columns: [cnt_agb], pk_columns: [] } + StreamMaterialize { columns: [cnt_agb], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count filter((t.a > t.b)))] } └─StreamGlobalSimpleAgg { aggs: [count, sum0(count filter((t.a > t.b)))] } └─StreamExchange { dist: Single } @@ -690,7 +690,7 @@ └─BatchSimpleAgg { aggs: [sum(t.v2) filter((t.v2 < 5:Int32))] } └─BatchScan { table: t, columns: [t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [b], pk_columns: [] } + StreamMaterialize { columns: [b], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v2) filter((t.v2 < 5:Int32)))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v2) filter((t.v2 < 5:Int32)))] } └─StreamExchange { dist: Single } @@ -711,7 +711,7 @@ └─BatchExchange { order: [], dist: HashShard(t.v1, t.v2, t.v3) } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [min, sum, t.v1(hidden), t.v3(hidden), t.v2(hidden)], pk_columns: [t.v1, t.v3, t.v2] } + StreamMaterialize { columns: [min, sum, t.v1(hidden), t.v3(hidden), t.v2(hidden)], pk_columns: [t.v1, t.v3, t.v2], pk_conflict: "no check" } └─StreamProject { exprs: [min(min(t.v3)), sum(sum(t.v1)), t.v1, t.v3, t.v2] } └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2], aggs: [count, min(min(t.v3)), sum(sum(t.v1))] } └─StreamExchange { dist: HashShard(t.v1, t.v3, t.v2) } @@ -730,7 +730,7 @@ └─BatchSimpleAgg { aggs: [min(t.v1), sum(t.v2)] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [min, sum], pk_columns: [] } + StreamMaterialize { columns: [min, sum], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [min(min(t.v1)), sum(sum(t.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), sum(sum(t.v2))] } └─StreamExchange { dist: Single } @@ -748,7 +748,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [min, sum], pk_columns: [] } + StreamMaterialize { columns: [min, sum], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [min(t.v1), sum(t.v2)] } └─StreamGlobalSimpleAgg { aggs: [count, min(t.v1), sum(t.v2)] } └─StreamExchange { dist: Single } @@ -778,9 +778,9 @@ └─BatchExchange { order: [], dist: HashShard(t.a, t.b) } └─BatchScan { table: t, columns: [t.a, t.b, t.c], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a, distinct_b_num, sum_c], pk_columns: [a] } - └─StreamProject { exprs: [t.a, count(distinct t.b), sum(t.c)] } - └─StreamHashAgg { group_key: [t.a], aggs: [count, count(distinct t.b), sum(t.c)] } + StreamMaterialize { columns: [a, distinct_b_num, sum_c], pk_columns: [a], pk_conflict: "no check" } + └─StreamProject { exprs: [t.a, count(t.b), sum(sum(t.c))] } + └─StreamHashAgg { group_key: [t.a], aggs: [count, count(t.b), sum(sum(t.c))] } └─StreamExchange { dist: HashShard(t.a) } └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: distinct agg and non-disintct agg with intersected argument @@ -801,9 +801,9 @@ └─BatchExpand { column_subsets: [[t.a, t.c], [t.a, t.b]] } └─BatchScan { table: t, columns: [t.a, t.b, t.c], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a, distinct_b_num, distinct_c_sum, sum_c], pk_columns: [a] } - └─StreamProject { exprs: [t.a, count(distinct t.b), count(distinct t.c), sum(t.c)] } - └─StreamHashAgg { group_key: [t.a], aggs: [count, count(distinct t.b), count(distinct t.c), sum(t.c)] } + StreamMaterialize { columns: [a, distinct_b_num, distinct_c_sum, sum_c], pk_columns: [a], pk_conflict: "no check" } + └─StreamProject { exprs: [t.a, count(t.b) filter((flag = 1:Int64)), count(t.c) filter((flag = 0:Int64)), sum(sum(t.c)) filter((flag = 0:Int64))] } + └─StreamHashAgg { group_key: [t.a], aggs: [count, count(t.b) filter((flag = 1:Int64)), count(t.c) filter((flag = 0:Int64)), sum(sum(t.c)) filter((flag = 0:Int64))] } └─StreamExchange { dist: HashShard(t.a) } └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: distinct agg with filter @@ -822,9 +822,9 @@ └─BatchExchange { order: [], dist: HashShard(t.a, t.b) } └─BatchScan { table: t, columns: [t.a, t.b, t.c], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a, count, sum], pk_columns: [a] } - └─StreamProject { exprs: [t.a, count(distinct t.b) filter((t.b < 100:Int32)), sum(t.c)] } - └─StreamHashAgg { group_key: [t.a], aggs: [count, count(distinct t.b) filter((t.b < 100:Int32)), sum(t.c)] } + StreamMaterialize { columns: [a, count, sum], pk_columns: [a], pk_conflict: "no check" } + └─StreamProject { exprs: [t.a, count(t.b) filter((count filter((t.b < 100:Int32)) > 0:Int64)), sum(sum(t.c))] } + └─StreamHashAgg { group_key: [t.a], aggs: [count, count(t.b) filter((count filter((t.b < 100:Int32)) > 0:Int64)), sum(sum(t.c))] } └─StreamExchange { dist: HashShard(t.a) } └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: non-distinct agg with filter @@ -849,7 +849,7 @@ └─LogicalProject { exprs: [t.b, (Length(t.a) * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: | - StreamMaterialize { columns: [s1], pk_columns: [] } + StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32)))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32)))] } └─StreamExchange { dist: Single } @@ -878,7 +878,7 @@ └─BatchSortAgg { group_key: [i.x], aggs: [count] } └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: | - StreamMaterialize { columns: [cnt, i.x(hidden)], pk_columns: [i.x] } + StreamMaterialize { columns: [cnt, i.x(hidden)], pk_columns: [i.x], pk_conflict: "no check" } └─StreamProject { exprs: [count, i.x] } └─StreamHashAgg { group_key: [i.x], aggs: [count, count] } └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } @@ -1073,21 +1073,21 @@ └─LogicalProject { exprs: [t.v1, (t.v1 * t.v1) as $expr1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } batch_plan: | - BatchProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Decimal::Float64, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / (sum0(count(t.v1)) - 1:Int64))::Float64, 0.5:Float64)) as $expr2, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr3] } + BatchProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Float64, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / (sum0(count(t.v1)) - 1:Int64))::Float64, 0.5:Float64)) as $expr2, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr3] } └─BatchSimpleAgg { aggs: [sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum($expr1), sum(t.v1), count(t.v1), sum($expr1), sum(t.v1), count(t.v1)] } └─BatchProject { exprs: [t.v1, (t.v1 * t.v1) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } batch_local_plan: | - BatchProject { exprs: [Case((count(t.v1) <= 1:Int64), null:Decimal::Float64, Pow(((sum($expr1)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / (count(t.v1) - 1:Int64))::Float64, 0.5:Float64)) as $expr2, Pow(((sum($expr1)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / count(t.v1))::Float64, 0.5:Float64) as $expr3] } + BatchProject { exprs: [Case((count(t.v1) <= 1:Int64), null:Float64, Pow(((sum($expr1)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / (count(t.v1) - 1:Int64))::Float64, 0.5:Float64)) as $expr2, Pow(((sum($expr1)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / count(t.v1))::Float64, 0.5:Float64) as $expr3] } └─BatchSimpleAgg { aggs: [sum($expr1), sum(t.v1), count(t.v1), sum($expr1), sum(t.v1), count(t.v1)] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v1, (t.v1 * t.v1) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [stddev_samp, stddev_pop], pk_columns: [] } - └─StreamProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Decimal::Float64, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / (sum0(count(t.v1)) - 1:Int64))::Float64, 0.5:Float64)) as $expr2, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr3] } + StreamMaterialize { columns: [stddev_samp, stddev_pop], pk_columns: [], pk_conflict: "no check" } + └─StreamProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Float64, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / (sum0(count(t.v1)) - 1:Int64))::Float64, 0.5:Float64)) as $expr2, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr3] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1))] } └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr1), sum(t.v1), count(t.v1), sum($expr1), sum(t.v1), count(t.v1)] } diff --git a/src/frontend/planner_test/tests/testdata/append_only.yaml b/src/frontend/planner_test/tests/testdata/append_only.yaml index fd187115815b5..fac78b65abd81 100644 --- a/src/frontend/planner_test/tests/testdata/append_only.yaml +++ b/src/frontend/planner_test/tests/testdata/append_only.yaml @@ -3,7 +3,7 @@ create table t1 (v1 int, v2 int) with (appendonly = true); select v1, max(v2) as mx2 from t1 group by v1; stream_plan: | - StreamMaterialize { columns: [v1, mx2], pk_columns: [v1] } + StreamMaterialize { columns: [v1, mx2], pk_columns: [v1], pk_conflict: "no check" } └─StreamProject { exprs: [t1.v1, max(t1.v2)] } └─StreamAppendOnlyHashAgg { group_key: [t1.v1], aggs: [count, max(t1.v2)] } └─StreamExchange { dist: HashShard(t1.v1) } @@ -13,7 +13,7 @@ create table t2 (v1 int, v3 int) with (appendonly = true); select t1.v1 as id, v2, v3 from t1 join t2 on t1.v1=t2.v1; stream_plan: | - StreamMaterialize { columns: [id, v2, v3, t1._row_id(hidden), t2._row_id(hidden), t2.v1(hidden)], pk_columns: [t1._row_id, t2._row_id, id, t2.v1] } + StreamMaterialize { columns: [id, v2, v3, t1._row_id(hidden), t2._row_id(hidden), t2.v1(hidden)], pk_columns: [t1._row_id, t2._row_id, id, t2.v1], pk_conflict: "no check" } └─StreamAppendOnlyHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v3, t1._row_id, t2._row_id, t2.v1] } ├─StreamExchange { dist: HashShard(t1.v1) } | └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } @@ -23,7 +23,7 @@ create table t1 (v1 int, v2 int) with (appendonly = true); select v1 from t1 order by v1 limit 3 offset 3; stream_plan: | - StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], order_descs: [v1, t1._row_id] } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], order_descs: [v1, t1._row_id], pk_conflict: "no check" } └─StreamAppendOnlyTopN { order: "[t1.v1 ASC]", limit: 3, offset: 3 } └─StreamExchange { dist: Single } └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } @@ -31,7 +31,7 @@ create table t1 (v1 int, v2 int) with (appendonly = true); select max(v1) as max_v1 from t1; stream_plan: | - StreamMaterialize { columns: [max_v1], pk_columns: [] } + StreamMaterialize { columns: [max_v1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(t1.v1))] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(max(t1.v1))] } └─StreamExchange { dist: Single } diff --git a/src/frontend/planner_test/tests/testdata/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index 46c5d783358d9..bc4bb13311ca9 100644 --- a/src/frontend/planner_test/tests/testdata/array.yaml +++ b/src/frontend/planner_test/tests/testdata/array.yaml @@ -4,13 +4,13 @@ logical_plan: | LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } batch_plan: | - BatchValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]] } + BatchValues { rows: [[ARRAY[foo, bar]:List { datatype: Varchar }]] } - sql: | values (ARRAY[1, 2+3, 4*5+1]); logical_plan: | LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Int32 }] } } batch_plan: | - BatchValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]] } + BatchValues { rows: [[ARRAY[1, 5, 21]:List { datatype: Int32 }]] } - sql: | create table t (v1 int); select (ARRAY[1, v1]) from t; @@ -52,21 +52,21 @@ LogicalProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32)) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchValues { rows: [[ArrayCat(Array(66:Int32), Array(123:Int32))]] } + BatchValues { rows: [[ARRAY[66, 123]:List { datatype: Int32 }]] } - sql: | select array_cat(array[array[66]], array[233]); logical_plan: | LogicalProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32)) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchValues { rows: [[ArrayCat(Array(Array(66:Int32)), Array(233:Int32))]] } + BatchValues { rows: [[ARRAY[{66}, {233}]:List { datatype: List { datatype: Int32 } }]] } - sql: | select array_cat(array[233], array[array[66]]); logical_plan: | LogicalProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32))) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchValues { rows: [[ArrayCat(Array(233:Int32), Array(Array(66:Int32)))]] } + BatchValues { rows: [[ARRAY[{233}, {66}]:List { datatype: List { datatype: Int32 } }]] } - sql: | select array_cat(array[233], array[array[array[66]]]); binder_error: 'Bind error: unable to find least restrictive type between integer[] and integer[][][]' @@ -82,7 +82,7 @@ LogicalProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchValues { rows: [[ArrayAppend(Array(66:Int32), 123:Int32)]] } + BatchValues { rows: [[ARRAY[66, 123]:List { datatype: Int32 }]] } - sql: | select array_append(123, 234); binder_error: 'Bind error: Cannot append integer to integer' @@ -98,7 +98,7 @@ LogicalProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32)) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchValues { rows: [[ArrayPrepend(123:Int32, Array(66:Int32))]] } + BatchValues { rows: [[ARRAY[123, 66]:List { datatype: Int32 }]] } - sql: | select array_prepend(123, 234); binder_error: 'Bind error: Cannot prepend integer to integer' diff --git a/src/frontend/planner_test/tests/testdata/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index 7320d46d79e9a..218e92f009f2c 100644 --- a/src/frontend/planner_test/tests/testdata/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/basic_query.yaml @@ -1,7 +1,7 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: values (11, 22), (33+(1+2), 44); batch_plan: | - BatchValues { rows: [[11:Int32, 22:Int32], [(33:Int32 + (1:Int32 + 2:Int32)), 44:Int32]] } + BatchValues { rows: [[11:Int32, 22:Int32], [36:Int32, 44:Int32]] } - sql: select * from t binder_error: 'Catalog error: table or source not found: t' - sql: | @@ -11,7 +11,7 @@ BatchExchange { order: [], dist: Single } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 bigint, v2 double precision); @@ -22,11 +22,11 @@ select * from t where 1>2 and 1=1 and 3<1 and 4<>1 or 1=1 and 2>=1 and 1<=2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } + └─BatchFilter { predicate: true:Boolean AND true:Boolean } └─BatchScan { table: t, columns: [], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } + StreamMaterialize { columns: [t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } + └─StreamFilter { predicate: true:Boolean AND true:Boolean } └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int); @@ -36,7 +36,7 @@ └─BatchFilter { predicate: (t.v1 < 1:Int32) } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamFilter { predicate: (t.v1 < 1:Int32) } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: test boolean expression common factor extraction @@ -94,7 +94,7 @@ BatchExchange { order: [], dist: Single } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: select 1 batch_plan: | @@ -129,11 +129,11 @@ - sql: | select * from unnest(Array[1,2,3]); batch_plan: | - BatchTableFunction { Unnest(Array(1:Int32, 2:Int32, 3:Int32)) } + BatchTableFunction { Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 }) } - sql: | select * from unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchTableFunction { Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32))) } + BatchTableFunction { Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } }) } - sql: | create table t1 (x int); select * from t1 where EXISTS(select * where t1.x=1); @@ -184,20 +184,20 @@ create materialized view mv(A,b) as select * from t; select a, b from mv; stream_plan: | - StreamMaterialize { columns: [a, b, mv.t._row_id(hidden)], pk_columns: [mv.t._row_id] } + StreamMaterialize { columns: [a, b, mv.t._row_id(hidden)], pk_columns: [mv.t._row_id], pk_conflict: "no check" } └─StreamTableScan { table: mv, columns: [mv.a, mv.b, mv.t._row_id], pk: [mv.t._row_id], dist: UpstreamHashShard(mv.t._row_id) } - sql: | create table t (v1 int, v2 int); create materialized view mv(a,b) as select v1+1,v2+1 from t; select * from mv; stream_plan: | - StreamMaterialize { columns: [a, b, mv.t._row_id(hidden)], pk_columns: [mv.t._row_id] } + StreamMaterialize { columns: [a, b, mv.t._row_id(hidden)], pk_columns: [mv.t._row_id], pk_conflict: "no check" } └─StreamTableScan { table: mv, columns: [mv.a, mv.b, mv.t._row_id], pk: [mv.t._row_id], dist: UpstreamHashShard(mv.t._row_id) } - sql: | create table t (id int primary key, col int); create index idx on t(col); select id from idx; stream_plan: | - StreamMaterialize { columns: [id], pk_columns: [id] } + StreamMaterialize { columns: [id], pk_columns: [id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(idx.id) } └─StreamTableScan { table: idx, columns: [idx.id], pk: [idx.id], dist: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/cast.yaml b/src/frontend/planner_test/tests/testdata/cast.yaml index 83c69c2134794..6dbb89aecc8db 100644 --- a/src/frontend/planner_test/tests/testdata/cast.yaml +++ b/src/frontend/planner_test/tests/testdata/cast.yaml @@ -18,7 +18,7 @@ sql: | select case when NULL then 1 end; batch_plan: | - BatchValues { rows: [[Case(null:Boolean, 1:Int32)]] } + BatchValues { rows: [[null:Int32]] } - name: implicit cast boolean (JOIN ON NULL) sql: | create table t1(v1 int); @@ -60,7 +60,7 @@ sql: | select case when 'y' then 1 end; batch_plan: | - BatchValues { rows: [[Case(true:Boolean, 1:Int32)]] } + BatchValues { rows: [[1:Int32]] } - name: implicit cast boolean (JOIN ON with literal 'y' of unknown type) sql: | create table t1(v1 int); diff --git a/src/frontend/planner_test/tests/testdata/column_pruning.yaml b/src/frontend/planner_test/tests/testdata/column_pruning.yaml index 24f0cc0ab3485..9b5181ef89560 100644 --- a/src/frontend/planner_test/tests/testdata/column_pruning.yaml +++ b/src/frontend/planner_test/tests/testdata/column_pruning.yaml @@ -150,6 +150,6 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t1, columns: [t1.a, t1.created_at], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_end] } + StreamMaterialize { columns: [a, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_end], pk_conflict: "no check" } └─StreamHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: [t1.a, window_end, t1._row_id] } └─StreamTableScan { table: t1, columns: [t1.a, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml b/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml index 09f890b0a09a1..e5241ef534cdd 100644 --- a/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml +++ b/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml @@ -8,7 +8,7 @@ └─LogicalProject { exprs: [t1.v1, t1.v2] } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | create table t1 (v1 int, v2 int); @@ -22,7 +22,7 @@ └─LogicalProject { exprs: [t1.v1] } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [v3, v4, v1, t2._row_id(hidden), t1._row_id(hidden)], pk_columns: [t2._row_id, t1._row_id, v3, v1] } + StreamMaterialize { columns: [v3, v4, v1, t2._row_id(hidden), t1._row_id(hidden)], pk_columns: [t2._row_id, t1._row_id, v3, v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t2._row_id, t1._row_id] } ├─StreamExchange { dist: HashShard(t2.v3) } | └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } @@ -40,7 +40,7 @@ └─LogicalProject { exprs: [t1.v1, t1.v2] } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | create table t1 (x int); diff --git a/src/frontend/planner_test/tests/testdata/delta_join.yaml b/src/frontend/planner_test/tests/testdata/delta_join.yaml index 331210d86a96c..4b00200aa87b0 100644 --- a/src/frontend/planner_test/tests/testdata/delta_join.yaml +++ b/src/frontend/planner_test/tests/testdata/delta_join.yaml @@ -8,7 +8,7 @@ /* should generate delta join plan, and stream index scan */ select * from a join b on a.a1 = b.b1 ; stream_plan: | - StreamMaterialize { columns: [a1, a2, b1, b2, i_a1.a._row_id(hidden), i_b1.b._row_id(hidden)], pk_columns: [i_a1.a._row_id, i_b1.b._row_id, a1, b1] } + StreamMaterialize { columns: [a1, a2, b1, b2, i_a1.a._row_id(hidden), i_b1.b._row_id(hidden)], pk_columns: [i_a1.a._row_id, i_b1.b._row_id, a1, b1], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(i_a1.a1, i_b1.b1, i_a1.a._row_id, i_b1.b._row_id) } └─StreamDeltaJoin { type: Inner, predicate: i_a1.a1 = i_b1.b1, output: [i_a1.a1, i_a1.a2, i_b1.b1, i_b1.b2, i_a1.a._row_id, i_b1.b._row_id] } ├─StreamIndexScan { index: i_a1, columns: [i_a1.a1, i_a1.a2, i_a1.a._row_id], pk: [i_a1.a._row_id], dist: UpstreamHashShard(i_a1.a1) } @@ -21,7 +21,7 @@ /* should generate delta join plan, and stream index scan */ select * from a join b on a.a1 = b.b1 ; stream_plan: | - StreamMaterialize { columns: [a1, a2, b1, b2, i_b1.b._row_id(hidden)], pk_columns: [a1, i_b1.b._row_id, b1] } + StreamMaterialize { columns: [a1, a2, b1, b2, i_b1.b._row_id(hidden)], pk_columns: [a1, i_b1.b._row_id, b1], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(a.a1, i_b1.b1, i_b1.b._row_id) } └─StreamDeltaJoin { type: Inner, predicate: a.a1 = i_b1.b1, output: all } ├─StreamTableScan { table: a, columns: [a.a1, a.a2], pk: [a.a1], dist: UpstreamHashShard(a.a1) } @@ -33,7 +33,7 @@ /* should generate delta join plan, and stream index scan */ select * from a join b on a.a1 = b.b1 ; stream_plan: | - StreamMaterialize { columns: [a1, a2, b1, b2], pk_columns: [a1, b1] } + StreamMaterialize { columns: [a1, a2, b1, b2], pk_columns: [a1, b1], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(a.a1, b.b1) } └─StreamDeltaJoin { type: Inner, predicate: a.a1 = b.b1, output: all } ├─StreamTableScan { table: a, columns: [a.a1, a.a2], pk: [a.a1], dist: UpstreamHashShard(a.a1) } diff --git a/src/frontend/planner_test/tests/testdata/distribution_derive.yaml b/src/frontend/planner_test/tests/testdata/distribution_derive.yaml index c8c3485113524..63867bf068b0d 100644 --- a/src/frontend/planner_test/tests/testdata/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/distribution_derive.yaml @@ -17,14 +17,14 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1, a.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1) } └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1] } ├─StreamIndexScan { index: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } └─StreamIndexScan { index: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([2, 3, 4, 5]) from 1 @@ -64,14 +64,14 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(ak1.k1) } └─BatchScan { table: ak1, columns: [ak1.k1, ak1.v], distribution: UpstreamHashShard(ak1.k1) } stream_plan: | - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1) } └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1] } ├─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } └─StreamIndexScan { index: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([2, 3, 4, 5]) from 1 @@ -111,14 +111,14 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1, a.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1) } └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1] } ├─StreamIndexScan { index: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } └─StreamTableScan { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([2, 3, 4, 5]) from 1 @@ -158,14 +158,14 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(ak1.k1) } └─BatchScan { table: ak1, columns: [ak1.k1, ak1.v], distribution: UpstreamHashShard(ak1.k1) } stream_plan: | - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1) } └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1] } ├─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } └─StreamTableScan { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([2, 3, 4, 5]) from 1 @@ -209,14 +209,14 @@ └─BatchExchange { order: [], dist: HashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1, a.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [max_v, a.k1(hidden)], pk_columns: [a.k1] } + StreamMaterialize { columns: [max_v, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } └─StreamProject { exprs: [max(a.v), a.k1] } └─StreamHashAgg { group_key: [a.k1], aggs: [count, max(a.v)] } └─StreamExchange { dist: HashShard(a.k1) } └─StreamTableScan { table: a, columns: [a.k1, a.v, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [max_v, a.k1(hidden)], pk_columns: [a.k1] } + StreamMaterialize { columns: [max_v, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(a.v), a.k1] } StreamHashAgg { group_key: [a.k1], aggs: [count, max(a.v)] } @@ -244,13 +244,13 @@ └─BatchSortAgg { group_key: [ak1.k1], aggs: [max(ak1.v)] } └─BatchScan { table: ak1, columns: [ak1.k1, ak1.v], distribution: UpstreamHashShard(ak1.k1) } stream_plan: | - StreamMaterialize { columns: [max_v, ak1.k1(hidden)], pk_columns: [ak1.k1] } + StreamMaterialize { columns: [max_v, ak1.k1(hidden)], pk_columns: [ak1.k1], pk_conflict: "no check" } └─StreamProject { exprs: [max(ak1.v), ak1.k1] } └─StreamHashAgg { group_key: [ak1.k1], aggs: [count, max(ak1.v)] } └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [max_v, ak1.k1(hidden)], pk_columns: [ak1.k1] } + StreamMaterialize { columns: [max_v, ak1.k1(hidden)], pk_columns: [ak1.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(ak1.v), ak1.k1] } StreamHashAgg { group_key: [ak1.k1], aggs: [count, max(ak1.v)] } @@ -276,14 +276,14 @@ └─BatchExchange { order: [ak1k2.k1 ASC], dist: HashShard(ak1k2.k1) } └─BatchScan { table: ak1k2, columns: [ak1k2.k1, ak1k2.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [max_v, ak1k2.k1(hidden)], pk_columns: [ak1k2.k1] } + StreamMaterialize { columns: [max_v, ak1k2.k1(hidden)], pk_columns: [ak1k2.k1], pk_conflict: "no check" } └─StreamProject { exprs: [max(ak1k2.v), ak1k2.k1] } └─StreamHashAgg { group_key: [ak1k2.k1], aggs: [count, max(ak1k2.v)] } └─StreamExchange { dist: HashShard(ak1k2.k1) } └─StreamTableScan { table: ak1k2, columns: [ak1k2.k1, ak1k2.v, ak1k2.k2, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [max_v, ak1k2.k1(hidden)], pk_columns: [ak1k2.k1] } + StreamMaterialize { columns: [max_v, ak1k2.k1(hidden)], pk_columns: [ak1k2.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(ak1k2.v), ak1k2.k1] } StreamHashAgg { group_key: [ak1k2.k1], aggs: [count, max(ak1k2.v)] } @@ -312,14 +312,14 @@ └─BatchExchange { order: [], dist: HashShard(ak1k2.k2) } └─BatchScan { table: ak1k2, columns: [ak1k2.k2, ak1k2.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [max_v, ak1k2.k2(hidden)], pk_columns: [ak1k2.k2] } + StreamMaterialize { columns: [max_v, ak1k2.k2(hidden)], pk_columns: [ak1k2.k2], pk_conflict: "no check" } └─StreamProject { exprs: [max(ak1k2.v), ak1k2.k2] } └─StreamHashAgg { group_key: [ak1k2.k2], aggs: [count, max(ak1k2.v)] } └─StreamExchange { dist: HashShard(ak1k2.k2) } └─StreamTableScan { table: ak1k2, columns: [ak1k2.k2, ak1k2.v, ak1k2.k1, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [max_v, ak1k2.k2(hidden)], pk_columns: [ak1k2.k2] } + StreamMaterialize { columns: [max_v, ak1k2.k2(hidden)], pk_columns: [ak1k2.k2], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(ak1k2.v), ak1k2.k2] } StreamHashAgg { group_key: [ak1k2.k2], aggs: [count, max(ak1k2.v)] } @@ -347,13 +347,13 @@ └─BatchSortAgg { group_key: [ak1k2.k1, ak1k2.k2], aggs: [sum(ak1k2.v)] } └─BatchScan { table: ak1k2, columns: [ak1k2.k1, ak1k2.k2, ak1k2.v], distribution: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } stream_plan: | - StreamMaterialize { columns: [sum_v, ak1k2.k1(hidden), ak1k2.k2(hidden)], pk_columns: [ak1k2.k1, ak1k2.k2] } + StreamMaterialize { columns: [sum_v, ak1k2.k1(hidden), ak1k2.k2(hidden)], pk_columns: [ak1k2.k1, ak1k2.k2], pk_conflict: "no check" } └─StreamProject { exprs: [sum(ak1k2.v), ak1k2.k1, ak1k2.k2] } └─StreamHashAgg { group_key: [ak1k2.k1, ak1k2.k2], aggs: [count, sum(ak1k2.v)] } └─StreamTableScan { table: ak1k2, columns: [ak1k2.k1, ak1k2.k2, ak1k2.v, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [sum_v, ak1k2.k1(hidden), ak1k2.k2(hidden)], pk_columns: [ak1k2.k1, ak1k2.k2] } + StreamMaterialize { columns: [sum_v, ak1k2.k1(hidden), ak1k2.k2(hidden)], pk_columns: [ak1k2.k1, ak1k2.k2], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(ak1k2.v), ak1k2.k1, ak1k2.k2] } StreamHashAgg { group_key: [ak1k2.k1, ak1k2.k2], aggs: [count, sum(ak1k2.v)] } @@ -377,13 +377,13 @@ └─BatchHashAgg { group_key: [ak1.k1, ak1.k2], aggs: [sum(ak1.v)] } └─BatchScan { table: ak1, columns: [ak1.k1, ak1.k2, ak1.v], distribution: UpstreamHashShard(ak1.k1) } stream_plan: | - StreamMaterialize { columns: [sum_v, ak1.k1(hidden), ak1.k2(hidden)], pk_columns: [ak1.k1, ak1.k2] } + StreamMaterialize { columns: [sum_v, ak1.k1(hidden), ak1.k2(hidden)], pk_columns: [ak1.k1, ak1.k2], pk_conflict: "no check" } └─StreamProject { exprs: [sum(ak1.v), ak1.k1, ak1.k2] } └─StreamHashAgg { group_key: [ak1.k1, ak1.k2], aggs: [count, sum(ak1.v)] } └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.k2, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [sum_v, ak1.k1(hidden), ak1.k2(hidden)], pk_columns: [ak1.k1, ak1.k2] } + StreamMaterialize { columns: [sum_v, ak1.k1(hidden), ak1.k2(hidden)], pk_columns: [ak1.k1, ak1.k2], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(ak1.v), ak1.k1, ak1.k2] } StreamHashAgg { group_key: [ak1.k1, ak1.k2], aggs: [count, sum(ak1.v)] } @@ -415,7 +415,7 @@ └─BatchExchange { order: [], dist: HashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1] } + StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } └─StreamProject { exprs: [max(count), a.k1] } └─StreamHashAgg { group_key: [a.k1], aggs: [count, max(count)] } └─StreamProject { exprs: [a.k1, count] } @@ -424,7 +424,7 @@ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1] } + StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(count), a.k1] } StreamHashAgg { group_key: [a.k1], aggs: [count, max(count)] } @@ -478,7 +478,7 @@ └─BatchExchange { order: [], dist: HashShard(a.k1, a.k2) } └─BatchScan { table: a, columns: [a.k1, a.k2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1] } + StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } └─StreamProject { exprs: [max(count), a.k1] } └─StreamHashAgg { group_key: [a.k1], aggs: [count, max(count)] } └─StreamExchange { dist: HashShard(a.k1) } @@ -488,7 +488,7 @@ └─StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1] } + StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(count), a.k1] } StreamHashAgg { group_key: [a.k1], aggs: [count, max(count)] } @@ -545,7 +545,7 @@ └─BatchExchange { order: [], dist: HashShard(a.k1, a.k2) } └─BatchScan { table: a, columns: [a.k1, a.k2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [max_num, a.k2(hidden)], pk_columns: [a.k2] } + StreamMaterialize { columns: [max_num, a.k2(hidden)], pk_columns: [a.k2], pk_conflict: "no check" } └─StreamProject { exprs: [max(count), a.k2] } └─StreamHashAgg { group_key: [a.k2], aggs: [count, max(count)] } └─StreamExchange { dist: HashShard(a.k2) } @@ -555,7 +555,7 @@ └─StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [max_num, a.k2(hidden)], pk_columns: [a.k2] } + StreamMaterialize { columns: [max_num, a.k2(hidden)], pk_columns: [a.k2], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(count), a.k2] } StreamHashAgg { group_key: [a.k2], aggs: [count, max(count)] } @@ -598,7 +598,7 @@ └─BatchExchange { order: [], dist: HashShard(a.k1, a.k2) } └─BatchScan { table: a, columns: [a.k1, a.k2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [max_num, a.k1(hidden), a.k2(hidden)], pk_columns: [a.k1, a.k2] } + StreamMaterialize { columns: [max_num, a.k1(hidden), a.k2(hidden)], pk_columns: [a.k1, a.k2], pk_conflict: "no check" } └─StreamProject { exprs: [max(count), a.k1, a.k2] } └─StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count, max(count)] } └─StreamProject { exprs: [a.k1, a.k2, count] } @@ -607,7 +607,7 @@ └─StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [max_num, a.k1(hidden), a.k2(hidden)], pk_columns: [a.k1, a.k2] } + StreamMaterialize { columns: [max_num, a.k1(hidden), a.k2(hidden)], pk_columns: [a.k1, a.k2], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(count), a.k1, a.k2] } StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count, max(count)] } @@ -647,7 +647,7 @@ └─BatchExchange { order: [], dist: HashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], pk_columns: [ak1.a._row_id, a.k1, ak1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], pk_columns: [ak1.a._row_id, a.k1, ak1.k1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } ├─StreamExchange { dist: HashShard(ak1.k1) } | └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } @@ -657,7 +657,7 @@ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], pk_columns: [ak1.a._row_id, a.k1, ak1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], pk_columns: [ak1.a._row_id, a.k1, ak1.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -703,7 +703,7 @@ └─BatchExchange { order: [], dist: HashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden), ak1.k1(hidden)], pk_columns: [a.k1, ak1.a._row_id, ak1.k1] } + StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden), ak1.k1(hidden)], pk_columns: [a.k1, ak1.a._row_id, ak1.k1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id, ak1.k1] } ├─StreamProject { exprs: [count, a.k1] } | └─StreamHashAgg { group_key: [a.k1], aggs: [count, count] } @@ -713,7 +713,7 @@ └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden), ak1.k1(hidden)], pk_columns: [a.k1, ak1.a._row_id, ak1.k1] } + StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden), ak1.k1(hidden)], pk_columns: [a.k1, ak1.a._row_id, ak1.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id, ak1.k1] } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -767,7 +767,7 @@ └─BatchExchange { order: [], dist: HashShard(b.k1) } └─BatchScan { table: b, columns: [b.k1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], pk_columns: [a.k1, b.k1] } + StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], pk_columns: [a.k1, b.k1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } ├─StreamProject { exprs: [count, a.k1] } | └─StreamHashAgg { group_key: [a.k1], aggs: [count, count] } @@ -779,7 +779,7 @@ └─StreamTableScan { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], pk_columns: [a.k1, b.k1] } + StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], pk_columns: [a.k1, b.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -824,12 +824,12 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t1, columns: [t1.row_id, t1.uid, t1.v, t1.created_at], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [row_id, uid, v, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start, window_end] } + StreamMaterialize { columns: [row_id, uid, v, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start, window_end], pk_conflict: "no check" } └─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] } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [row_id, uid, v, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start, window_end] } + StreamMaterialize { columns: [row_id, uid, v, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start, window_end], pk_conflict: "no check" } 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] } 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) } diff --git a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml index a7843a952cfc5..1f9f5efe75d81 100644 --- a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml @@ -14,7 +14,7 @@ └─LogicalAgg { aggs: [max(t2.v2)] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | - StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (t1.v1 > max(max(t2.v2))), output: [t1.v1, t1._row_id] } ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } @@ -56,7 +56,7 @@ └─LogicalAgg { aggs: [max(t2.v2)] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | - StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t1.v1, t1._row_id] } └─StreamDynamicFilter { predicate: ($expr1 > max(max(t2.v2))), output: [t1.v1, $expr1, t1._row_id] } ├─StreamProject { exprs: [t1.v1, (t1.v1 + t1.v1) as $expr1, t1._row_id] } @@ -109,7 +109,7 @@ └─LogicalAgg { aggs: [max(t2.v2)] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | - StreamMaterialize { columns: [v1, max, t1._row_id(hidden)], pk_columns: [t1._row_id, v1, max] } + StreamMaterialize { columns: [v1, max, t1._row_id(hidden)], pk_columns: [t1._row_id, v1, max], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = max(max(t2.v2)), output: [t1.v1, max(max(t2.v2)), t1._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } | └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } @@ -132,7 +132,7 @@ └─LogicalAgg { aggs: [max(t2.v2)] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | - StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t1.v1, t1._row_id] } └─StreamDynamicFilter { predicate: ($expr1 > max(max(t2.v2))), output: [t1.v1, $expr1, t1._row_id] } ├─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } @@ -150,7 +150,7 @@ create table t2 (v2 int); with max_v2 as (select max(v2) max from t2) select v1 from t1 where exists (select * from max_v2 where v1 > max); stream_plan: | - StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (t1.v1 > max(max(t2.v2))), output: [t1.v1, t1._row_id] } ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } @@ -172,7 +172,7 @@ └─LogicalAgg { aggs: [max(t2.v2)] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | - StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (t1.v1 > $expr2), output: [t1.v1, t1._row_id] } ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index 0aca7036a4366..54561c668b574 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -42,6 +42,14 @@ LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } + Inline Session Timezone: + + BatchValues { rows: [[1:Int32]] } + + Const eval exprs: + + BatchValues { rows: [[1:Int32]] } + To Batch Physical Plan: BatchValues { rows: [[1:Int32]] } @@ -55,7 +63,7 @@ "stages": { "0": { "root": { - "plan_node_id": 30, + "plan_node_id": 34, "plan_node_type": "BatchValues", "schema": [ { @@ -168,7 +176,7 @@ - sql: | explain create table t (v1 int, v2 varchar); explain_output: | - StreamMaterialize { columns: [v1, v2, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [v1, v2, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "overwrite" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamRowIdGen { row_id_index: 2 } └─StreamDml { columns: [v1, v2, _row_id] } @@ -176,7 +184,7 @@ - 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' ) row format json; explain_output: | - StreamMaterialize { columns: [v1, v2, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [v1, v2, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "overwrite" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamRowIdGen { row_id_index: 2 } └─StreamDml { columns: [v1, v2, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 819bfad027f93..35fa5b38c1726 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -19,17 +19,17 @@ - sql: | values(cast(1 as bigint)); batch_plan: | - BatchValues { rows: [[1:Int32::Int64]] } + BatchValues { rows: [[1:Int64]] } - sql: | values(not true); batch_plan: | - BatchValues { rows: [[Not(true:Boolean)]] } + BatchValues { rows: [[false:Boolean]] } - sql: | create table t (); select (((((false is not true) is true) is not false) is false) is not null) is null from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [IsNull(IsNotNull(IsFalse(IsNotFalse(IsTrue(IsNotTrue(false:Boolean)))))) as $expr1] } + └─BatchProject { exprs: [false:Boolean] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: bind between sql: | @@ -65,12 +65,12 @@ sql: | SELECT 1::real in (3, 1.0, 2); batch_plan: | - BatchValues { rows: [[In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32)]] } + BatchValues { rows: [[true:Boolean]] } - name: not in-list with aligned types sql: | SELECT 1::real not in (3, 1.0, 2); batch_plan: | - BatchValues { rows: [[Not(In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32))]] } + BatchValues { rows: [[false:Boolean]] } - name: in-list with misaligned types sql: | SELECT true in (3, 1.0, 2); @@ -80,7 +80,7 @@ create table t (v1 int); SELECT 1 in (3, 0.5*2, min(v1)) from t; batch_plan: | - BatchProject { exprs: [(In(1:Int32::Decimal, 3:Int32::Decimal, (0.5:Decimal * 2:Int32)) OR (1:Int32 = min(min(t.v1)))) as $expr1] } + BatchProject { exprs: [(true:Boolean OR (1:Int32 = min(min(t.v1)))) as $expr1] } └─BatchSimpleAgg { aggs: [min(min(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1)] } @@ -92,7 +92,7 @@ SELECT b2 from b where 1 in (3, 1.0, (select min(v1) from t)); batch_plan: | BatchProject { exprs: [b.b2] } - └─BatchFilter { predicate: (In(1:Int32::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (1:Int32 = min(min(t.v1)))) } + └─BatchFilter { predicate: (true:Boolean OR (1:Int32 = min(min(t.v1)))) } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b2], distribution: SomeShard } @@ -106,7 +106,7 @@ create table b (b1 int, b2 int); SELECT b2 from b where exists (select 2 from t where v1 in (3, 1.0, b1)); batch_plan: | - BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } + BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b1, b.b2], distribution: SomeShard } └─BatchExchange { order: [], dist: Single } @@ -119,19 +119,19 @@ - sql: | values(round(42.4382, 2)); batch_plan: | - BatchValues { rows: [[RoundDigit(42.4382:Decimal, 2:Int32)]] } + BatchValues { rows: [[42.44:Decimal]] } - sql: | values(round(42.4382)); batch_plan: | - BatchValues { rows: [[Round(42.4382:Decimal)]] } + BatchValues { rows: [[42:Decimal]] } - sql: | values(round(42)); batch_plan: | - BatchValues { rows: [[Round(42:Int32::Float64)]] } + BatchValues { rows: [[42:Float64]] } - sql: | values(round(42, 2)); batch_plan: | - BatchValues { rows: [[RoundDigit(42:Int32::Decimal, 2:Int32)]] } + BatchValues { rows: [[42:Decimal]] } - sql: | values(round(true)); binder_error: |- @@ -141,18 +141,18 @@ -- Single quoted literal can be treated as number without error. values(round('123')); batch_plan: | - BatchValues { rows: [[Round(123:Float64)]] } + BatchValues { rows: [[123:Float64]] } - sql: | -- When it is invalid, PostgreSQL reports error during explain, but we have to wait until execution as of now. #4235 values(round('abc')); - sql: | values(extract(hour from timestamp '2001-02-16 20:38:40')); batch_plan: | - BatchValues { rows: [[Extract('HOUR':Varchar, '2001-02-16 20:38:40':Timestamp)]] } + BatchValues { rows: [[20:Decimal]] } - sql: | values('Postgres' not like 'Post%'); batch_plan: | - BatchValues { rows: [[Not(Like('Postgres':Varchar, 'Post%':Varchar))]] } + BatchValues { rows: [[false:Boolean]] } - sql: | values(1 not like 1.23); binder_error: |- @@ -161,12 +161,12 @@ - sql: | select length(trim(trailing '1' from '12'))+length(trim(leading '2' from '23'))+length(trim(both '3' from '34')); batch_plan: | - BatchValues { rows: [[((Length(Rtrim('12':Varchar, '1':Varchar)) + Length(Ltrim('23':Varchar, '2':Varchar))) + Length(Trim('34':Varchar, '3':Varchar)))]] } + BatchValues { rows: [[4:Int32]] } - sql: | select position(replace('1','1','2'),'123') where '12' like '%1'; batch_plan: | - BatchProject { exprs: [Position(Replace('1':Varchar, '1':Varchar, '2':Varchar), '123':Varchar) as $expr1] } - └─BatchFilter { predicate: Like('12':Varchar, '%1':Varchar) } + BatchProject { exprs: [0:Int32] } + └─BatchFilter { predicate: false:Boolean } └─BatchValues { rows: [[]] } - name: case searched form with else sql: | @@ -174,11 +174,11 @@ select (case when v1=1 then 1 when v1=2 then 2 else 0.0 end) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal) as $expr1] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal) as $expr1, t._row_id] } + StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: case searched form without else sql: | @@ -186,7 +186,7 @@ select (case when v1=1 then 1 when v1=2 then 2.1 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2.1:Decimal) as $expr1] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2.1:Decimal) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case simple form sql: | @@ -194,7 +194,7 @@ select (case v1 when 1 then 1 when 2.0 then 2 else 0.0 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2.0:Decimal), 2:Int32::Decimal, 0.0:Decimal) as $expr1] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2.0:Decimal), 2:Decimal, 0.0:Decimal) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case misaligned result types sql: | @@ -216,7 +216,7 @@ └─BatchProject { exprs: [Case((t.v1 = 1:Int32), null:Int32, t.v1) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [Case((t.v1 = 1:Int32), null:Int32, t.v1) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | @@ -237,7 +237,7 @@ └─BatchProject { exprs: [Coalesce(t.v1, 1:Int32) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [Coalesce(t.v1, 1:Int32) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | @@ -260,18 +260,18 @@ select concat_ws(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar) as $expr1] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1':Varchar) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar) as $expr1, t._row_id] } + StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } + └─StreamProject { exprs: [ConcatWs(t.v1, '1':Varchar) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 varchar); select concat_ws(v1, 1.2) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1.2:Decimal::Varchar) as $expr1] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1.2':Varchar) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -286,11 +286,11 @@ select concat(v1, v2, v3, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar) as $expr1] } + └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar) as $expr1] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar) as $expr1, t._row_id] } + StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } + └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 float); @@ -306,19 +306,19 @@ - sql: | select concat(':', true); batch_plan: | - BatchValues { rows: [[ConcatWs('':Varchar, ':':Varchar, BoolOut(true:Boolean))]] } + BatchValues { rows: [[':t':Varchar]] } - sql: | select ':' || true; batch_plan: | - BatchValues { rows: [[ConcatOp(':':Varchar, true:Boolean::Varchar)]] } + BatchValues { rows: [[':true':Varchar]] } - sql: | select substr('hello', NULL); batch_plan: | - BatchValues { rows: [[Substr('hello':Varchar, null:Int32)]] } + BatchValues { rows: [[null:Varchar]] } - sql: | select substr(NULL, 1); batch_plan: | - BatchValues { rows: [[Substr(null:Varchar, 1:Int32)]] } + BatchValues { rows: [[null:Varchar]] } - sql: | select pg_typeof('123'); batch_plan: | @@ -406,10 +406,10 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr35, Array(2:Int32)))) as $expr1] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr35, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchValues { rows: [[Array(1:Int32)]] } + └─BatchValues { rows: [[ARRAY[1]:List { datatype: Int32 }]] } - sql: | select 1 < ALL(array[null]::integer[]); logical_plan: | @@ -429,10 +429,10 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat($expr35, Array(2:Int32)))) as $expr1] } + BatchProject { exprs: [All((1:Int32 < ArrayCat($expr35, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchValues { rows: [[Array(1:Int32)]] } + └─BatchValues { rows: [[ARRAY[1]:List { datatype: Int32 }]] } - name: now expression sql: | create table t (v1 timestamp with time zone); @@ -442,7 +442,7 @@ └─LogicalFilter { predicate: (t.v1 >= Now) } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: | - StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (t.v1 >= now), output: [t.v1, t._row_id] } ├─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: Broadcast } @@ -456,18 +456,18 @@ └─LogicalFilter { predicate: (t.v1 >= (Now - '00:00:02':Interval)) } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: | - StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (t.v1 >= $expr1), output: [t.v1, t._row_id] } ├─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(now - '00:00:02':Interval) as $expr1], watermark_columns: [(now - '00:00:02':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval)] } └─StreamNow { output: [now] } - name: and of two now expression condition sql: | create table t (v1 timestamp with time zone, v2 timestamp with time zone); select * from t where v1 >= now() and v2 >= now(); stream_plan: | - StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (t.v2 >= now), output: [t.v1, t.v2, t._row_id] } ├─StreamDynamicFilter { predicate: (t.v1 >= now), output: [t.v1, t.v2, t._row_id] } | ├─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -485,7 +485,7 @@ create table t (v1 timestamp with time zone, v2 int); select max(v1) as max_time from t group by v2 having max(v1) >= now(); stream_plan: | - StreamMaterialize { columns: [max_time, t.v2(hidden)], pk_columns: [t.v2] } + StreamMaterialize { columns: [max_time, t.v2(hidden)], pk_columns: [t.v2], pk_conflict: "no check" } └─StreamProject { exprs: [max(t.v1), t.v2] } └─StreamDynamicFilter { predicate: (max(t.v1) >= now), output: [max(t.v1), max(t.v1), t.v2] } ├─StreamProject { exprs: [max(t.v1), max(t.v1), t.v2] } @@ -525,14 +525,15 @@ sql: | select false >= 'LN1O0QP1yi' NOT IN (md5('4SeUPZhUbH')) batch_plan: | - BatchValues { rows: [[(false:Boolean >= Not(In('LN1O0QP1yi':Varchar, Md5('4SeUPZhUbH':Varchar))))]] } + BatchValues { rows: [[false:Boolean]] } - name: const_eval of const expr sql: | create table t(v1 int); select 1 + 2 + v1 from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [((1:Int32 + 2:Int32) + t.v1) as $expr1] } + └─BatchProject { exprs: [(3:Int32 + t.v1) as $expr1] } └─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' diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index ca394a435975b..a227d2992cd58 100644 --- a/src/frontend/planner_test/tests/testdata/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/index_selection.yaml @@ -77,7 +77,7 @@ select a,b from t1 where a in (1,2) and b in (2,3) batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: In(idx1.b, 2:Int32::Decimal, 3:Int32::Decimal) } + └─BatchFilter { predicate: In(idx1.b, 2:Decimal, 3:Decimal) } └─BatchScan { table: idx1, columns: [idx1.a, idx1.b], scan_ranges: [idx1.a = Int32(1) , idx1.a = Int32(2)], distribution: UpstreamHashShard(idx1.a, idx1.b) } - sql: | create table t1 (a int, b numeric, c bigint); @@ -213,13 +213,13 @@ update t1 set c = 3 where a = 1 and b = 2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } + └─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] } └─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:Int32::Int64, $3] } + 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] } └─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 } @@ -613,7 +613,7 @@ create table t1 (a int primary key); select * from t1 order by a limit 1; stream_plan: | - StreamMaterialize { columns: [a], pk_columns: [a] } + StreamMaterialize { columns: [a], pk_columns: [a], pk_conflict: "no check" } └─StreamProject { exprs: [t1.a] } └─StreamTopN { order: "[t1.a ASC]", limit: 1, offset: 0 } └─StreamExchange { dist: Single } diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index 170089b02a1b3..423f8e0e97f7e 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -14,7 +14,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[22.33:Decimal::Float32, 33:Int32], [44:Int32::Float32, 55.0:Decimal::Int32]] } + └─BatchValues { rows: [[22.33:Float32, 33:Int32], [44:Float32, 55:Int32]] } - name: prohibit inserting different number of nulls per row sql: | create table t (v1 real, v2 int, v3 varchar); @@ -27,7 +27,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[1:Int32::Float32, 2:Int32, null:Varchar]] } + └─BatchValues { rows: [[1:Float32, 2:Int32, null:Varchar]] } - name: insert values with implicit null (multiple rows) sql: | create table t (v1 real, v2 int, v3 varchar); @@ -35,7 +35,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[1:Int32::Float32, 2:Int32, null:Varchar], [3:Int32::Float32, 4:Int32, null:Varchar]] } + └─BatchValues { rows: [[1:Float32, 2:Int32, null:Varchar], [3:Float32, 4:Int32, null:Varchar]] } - name: implicit null user defined columns 1 sql: | create table t (v1 int, v2 int); @@ -129,7 +129,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [['2020-01-01 01:02:03':Timestamp::Time], ['03:04:05':Time]] } + └─BatchValues { rows: [['01:02:03':Time], ['03:04:05':Time]] } - name: a `VALUES` without insert context may be invalid on its own (compare with above) sql: | create table t (v1 time); @@ -149,7 +149,7 @@ sql: | values (1), (null), (2.3); batch_plan: | - BatchValues { rows: [[1:Int32::Decimal], [null:Decimal], [2.3:Decimal]] } + BatchValues { rows: [[1:Decimal], [null:Decimal], [2.3:Decimal]] } - name: rows of different number of columns sql: | values (1), (2, 3); @@ -171,7 +171,7 @@ BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: ['2020-01-01 01:02:03':Timestamp::Time as $expr1, 11:Int32, 4.5:Decimal::Float32 as $expr2] } + └─BatchProject { exprs: ['01:02:03':Time, 11:Int32, 4.5:Float32] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: insert into select with cast error sql: | diff --git a/src/frontend/planner_test/tests/testdata/join.yaml b/src/frontend/planner_test/tests/testdata/join.yaml index eec71f23fcdc5..b96a8e504c8ce 100644 --- a/src/frontend/planner_test/tests/testdata/join.yaml +++ b/src/frontend/planner_test/tests/testdata/join.yaml @@ -13,7 +13,7 @@ | └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } └─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] } stream_plan: | - StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, v3, t3._row_id, v5] } + StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, v3, t3._row_id, v5], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = t3.v5, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] } ├─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } | ├─StreamExchange { dist: HashShard(t1.v1) } @@ -32,7 +32,7 @@ ├─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } stream_plan: | - StreamMaterialize { columns: [t1v1, t2v1, t._row_id(hidden), t._row_id#1(hidden)], pk_columns: [t._row_id, t._row_id#1, t1v1, t2v1] } + StreamMaterialize { columns: [t1v1, t2v1, t._row_id(hidden), t._row_id#1(hidden)], pk_columns: [t._row_id, t._row_id#1, t1v1, t2v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v1, t._row_id, t._row_id] } ├─StreamExchange { dist: HashShard(t.v1) } | └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -64,7 +64,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t3, columns: [t3.v1, t3.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t3_v1, t3_v2, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t1_v1, t2_v1, t3._row_id, t2_v2, t3_v2] } + StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t3_v1, t3_v2, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t1_v1, t2_v1, t3._row_id, t2_v2, t3_v2], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t2.v2 = t3.v2, output: [t1.v1, t1.v2, t2.v1, t2.v2, t3.v1, t3.v2, t1._row_id, t2._row_id, t3._row_id] } ├─StreamExchange { dist: HashShard(t2.v2) } | └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } @@ -92,7 +92,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t2, columns: [t2.v1, t2.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [t1_v2, t2_v2, t1._row_id(hidden), t1.v1(hidden), t2._row_id(hidden), t2.v1(hidden)], pk_columns: [t1._row_id, t2._row_id, t1.v1, t2.v1] } + StreamMaterialize { columns: [t1_v2, t2_v2, t1._row_id(hidden), t1.v1(hidden), t2._row_id(hidden), t2.v1(hidden)], pk_columns: [t1._row_id, t2._row_id, t1.v1, t2.v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v2, t2.v2, t1._row_id, t1.v1, t2._row_id, t2.v1] } ├─StreamExchange { dist: HashShard(t1.v1) } | └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } @@ -153,7 +153,7 @@ └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: all } └─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)], pk_columns: [i.t._row_id, i.t._row_id#1, ix, iix] } + StreamMaterialize { columns: [ix, iix, i.t._row_id(hidden), i.t._row_id#1(hidden)], pk_columns: [i.t._row_id, i.t._row_id#1, ix, iix], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id] } ├─StreamExchange { dist: HashShard(i.x) } | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } @@ -169,7 +169,7 @@ └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: all } └─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)], pk_columns: [i.t._row_id, t._row_id, ix, tx] } + StreamMaterialize { columns: [ix, tx, i.t._row_id(hidden), t._row_id(hidden)], pk_columns: [i.t._row_id, t._row_id, ix, tx], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: i.x = t.x, output: [i.x, t.x, i.t._row_id, t._row_id] } ├─StreamExchange { dist: HashShard(i.x) } | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } @@ -195,7 +195,7 @@ └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: [i.x] } └─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.x#1(hidden), i.t._row_id#2(hidden), i.t._row_id#3(hidden), i.x#2(hidden), i.x#3(hidden)], pk_columns: [i.t._row_id, i.t._row_id#1, i.x, i.x#1, i.t._row_id#2, i.t._row_id#3, i.x#2, i.x#3] } + StreamMaterialize { columns: [x, i.t._row_id(hidden), i.t._row_id#1(hidden), i.x(hidden), i.x#1(hidden), i.t._row_id#2(hidden), i.t._row_id#3(hidden), i.x#2(hidden), i.x#3(hidden)], pk_columns: [i.t._row_id, i.t._row_id#1, i.x, i.x#1, i.t._row_id#2, i.t._row_id#3, i.x#2, i.x#3], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x) } └─StreamProject { exprs: [Coalesce(i.x, i.x) as $expr1, i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x] } └─StreamHashJoin { type: FullOuter, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.t._row_id, i.t._row_id, i.x] } @@ -499,7 +499,7 @@ └─BatchExchange { order: [], dist: HashShard(b.x) } └─BatchScan { table: b, columns: [b.x], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [y, z, $expr144(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], pk_columns: [a._row_id, b._row_id, a.x, b.x], order_descs: [$expr144, a._row_id, b._row_id, a.x, b.x] } + StreamMaterialize { columns: [y, z, $expr156(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], pk_columns: [a._row_id, b._row_id, a.x, b.x], order_descs: [$expr156, a._row_id, b._row_id, a.x, b.x], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(a._row_id, b._row_id, a.x, b.x) } └─StreamProject { exprs: [(2:Int32 * Coalesce(a.x, b.x)) as $expr1, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr2, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr3, a._row_id, b._row_id, a.x, b.x] } └─StreamFilter { predicate: ((2:Int32 * Coalesce(a.x, b.x)) < 10:Int32) } @@ -600,7 +600,7 @@ └─BatchExchange { order: [], dist: HashShard(t2.v2) } └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr25(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr25, v2] } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr29(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr29, v2], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] } ├─StreamExchange { dist: HashShard($expr1) } | └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/mv_column_name.yaml b/src/frontend/planner_test/tests/testdata/mv_column_name.yaml index d11c80d3d0240..9c8dce79ff211 100644 --- a/src/frontend/planner_test/tests/testdata/mv_column_name.yaml +++ b/src/frontend/planner_test/tests/testdata/mv_column_name.yaml @@ -14,7 +14,7 @@ create table t (a int); select a is null as is_null from t; stream_plan: | - StreamMaterialize { columns: [is_null, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [is_null, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [IsNull(t.a) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: is_true with alias @@ -22,7 +22,7 @@ create table t (a bool); select a, a is true as is_true from t; stream_plan: | - StreamMaterialize { columns: [a, is_true, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [a, is_true, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t.a, IsTrue(t.a) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: column name specified more than once @@ -53,7 +53,7 @@ create table t (a int); select count(*), max(a) from t; stream_plan: | - StreamMaterialize { columns: [count, max], pk_columns: [] } + StreamMaterialize { columns: [count, max], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count), max(max(t.a))] } └─StreamGlobalSimpleAgg { aggs: [count, sum0(count), max(max(t.a))] } └─StreamExchange { dist: Single } diff --git a/src/frontend/planner_test/tests/testdata/mv_on_mv.yaml b/src/frontend/planner_test/tests/testdata/mv_on_mv.yaml index 8c1a17290a1dd..bc4d8954a83f5 100644 --- a/src/frontend/planner_test/tests/testdata/mv_on_mv.yaml +++ b/src/frontend/planner_test/tests/testdata/mv_on_mv.yaml @@ -11,7 +11,7 @@ sql: | select m1.v1 as m1v1, m1.v2 as m1v2, m2.v1 as m2v1, m2.v2 as m2v2 from m1 join m2 on m1.v1 = m2.v1; stream_plan: | - StreamMaterialize { columns: [m1v1, m1v2, m2v1, m2v2, m1.t1._row_id(hidden), m2.t1._row_id(hidden)], pk_columns: [m1.t1._row_id, m2.t1._row_id, m1v1, m2v1] } + StreamMaterialize { columns: [m1v1, m1v2, m2v1, m2v2, m1.t1._row_id(hidden), m2.t1._row_id(hidden)], pk_columns: [m1.t1._row_id, m2.t1._row_id, m1v1, m2v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: m1.v1 = m2.v1, output: [m1.v1, m1.v2, m2.v1, m2.v2, m1.t1._row_id, m2.t1._row_id] } ├─StreamExchange { dist: HashShard(m1.v1) } | └─StreamTableScan { table: m1, columns: [m1.v1, m1.v2, m1.t1._row_id], pk: [m1.t1._row_id], dist: UpstreamHashShard(m1.t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index ed9434c620b06..78340b294cea7 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -43,11 +43,11 @@ BatchExchange { order: [], dist: Single } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } 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) } Upstream @@ -69,14 +69,14 @@ └─BatchProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr1, bid.date_time] } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr1, bid.date_time, 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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr48, bid.date_time, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr52, 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) } Upstream BatchPlanNode @@ -91,12 +91,12 @@ └─BatchFilter { 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)) } └─BatchScan { table: bid, columns: [bid.auction, bid.price], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, price, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, price, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } └─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)) } └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, price, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } 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) } @@ -122,7 +122,7 @@ └─BatchFilter { predicate: (auction.category = 10:Int32) } └─BatchScan { table: auction, columns: [auction.id, auction.seller, auction.category], distribution: UpstreamHashShard(auction.id) } stream_plan: | - StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], pk_columns: [id, person.id, auction.seller] } + StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], pk_columns: [id, person.id, auction.seller], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } ├─StreamExchange { dist: HashShard(auction.seller) } | └─StreamProject { exprs: [auction.id, auction.seller] } @@ -133,7 +133,7 @@ └─StreamTableScan { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], pk_columns: [id, person.id, auction.seller] } + StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], pk_columns: [id, person.id, auction.seller], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -186,7 +186,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [category, avg], pk_columns: [category] } + StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } └─StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr1] } └─StreamHashAgg { group_key: [auction.category], aggs: [count, sum(max(bid.price)), count(max(bid.price))] } └─StreamExchange { dist: HashShard(auction.category) } @@ -201,9 +201,9 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [category, avg], pk_columns: [category] } + StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr49] } + StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr53] } StreamHashAgg { group_key: [auction.category], aggs: [count, sum(max(bid.price)), count(max(bid.price))] } result table: 0, state tables: [] StreamExchange Hash([1]) from 1 @@ -288,7 +288,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [window_start, auction, window_start#1] } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [window_start, auction, window_start#1], pk_conflict: "no check" } └─StreamProject { exprs: [bid.auction, count, window_start, window_start] } └─StreamFilter { predicate: (count >= max(count)) } └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } @@ -308,7 +308,7 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [window_start, auction, window_start#1] } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [window_start, auction, window_start#1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [bid.auction, count, window_start, window_start] } StreamFilter { predicate: (count >= max(count)) } @@ -408,7 +408,7 @@ └─BatchProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, bid.price] } └─BatchScan { table: bid, columns: [bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr103(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr103, price, max(bid.price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr112(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr112, price, max(bid.price)], pk_conflict: "no check" } └─StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr1, max(bid.price)] } └─StreamFilter { predicate: (bid.date_time >= $expr2) AND (bid.date_time <= $expr1) } └─StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } @@ -422,10 +422,10 @@ └─StreamTableScan { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr103(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr103, price, max(bid.price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr112(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr112, price, max(bid.price)], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr103, max(bid.price)] } - StreamFilter { predicate: (bid.date_time >= $expr105) AND (bid.date_time <= $expr103) } + StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr112, max(bid.price)] } + StreamFilter { predicate: (bid.date_time >= $expr113) AND (bid.date_time <= $expr112) } StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([2]) from 1 @@ -437,23 +437,23 @@ BatchPlanNode Fragment 2 - StreamProject { exprs: [max(bid.price), $expr103, ($expr103 - '00:00:10':Interval) as $expr105] } - StreamAppendOnlyHashAgg { group_key: [$expr103], aggs: [count, max(bid.price)] } + StreamProject { exprs: [max(bid.price), $expr112, ($expr112 - '00:00:10':Interval) as $expr113] } + StreamAppendOnlyHashAgg { group_key: [$expr112], aggs: [count, max(bid.price)] } result table: 4, state tables: [] StreamExchange Hash([0]) from 3 Fragment 3 - StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr103, bid.price, bid._row_id] } + StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr112, 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) } Upstream BatchPlanNode Table 0 { columns: [bid_auction, bid_bidder, bid_price, bid_date_time, bid__row_id], primary key: [$2 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } Table 1 { columns: [bid_price, bid__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 2 { columns: [max(bid_price), $expr103, $expr105], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 3 { columns: [max(bid_price), $expr103, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 4 { columns: [$expr103, count, max(bid_price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr103, max(bid.price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 2 { columns: [max(bid_price), $expr112, $expr113], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(bid_price), $expr112, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 4 { columns: [$expr112, count, max(bid_price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr112, max(bid.price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - id: nexmark_q8 before: - create_tables @@ -502,7 +502,7 @@ └─BatchProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4] } └─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [id, name, starttime, $expr200(hidden), auction.seller(hidden), $expr197(hidden), $expr198(hidden)], pk_columns: [id, name, starttime, $expr200, auction.seller, $expr197, $expr198] } + StreamMaterialize { columns: [id, name, starttime, $expr214(hidden), auction.seller(hidden), $expr215(hidden), $expr216(hidden)], pk_columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } | └─StreamProject { exprs: [person.id, person.name, $expr1, $expr2] } @@ -516,38 +516,38 @@ └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr200(hidden), auction.seller(hidden), $expr197(hidden), $expr198(hidden)], pk_columns: [id, name, starttime, $expr200, auction.seller, $expr197, $expr198] } + StreamMaterialize { columns: [id, name, starttime, $expr214(hidden), auction.seller(hidden), $expr215(hidden), $expr216(hidden)], pk_columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216], pk_conflict: "no check" } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr199 = $expr197 AND $expr200 = $expr198, output: all } + StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr213 = $expr215 AND $expr214 = $expr216, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0, 2, 3]) from 1 - StreamProject { exprs: [auction.seller, $expr197, $expr198] } - StreamHashAgg { group_key: [auction.seller, $expr197, $expr198], aggs: [count] } + StreamProject { exprs: [auction.seller, $expr215, $expr216] } + StreamHashAgg { group_key: [auction.seller, $expr215, $expr216], aggs: [count] } result table: 5, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - StreamProject { exprs: [person.id, person.name, $expr199, $expr200] } - StreamHashAgg { group_key: [person.id, person.name, $expr199, $expr200], aggs: [count] } + StreamProject { exprs: [person.id, person.name, $expr213, $expr214] } + StreamHashAgg { group_key: [person.id, person.name, $expr213, $expr214], aggs: [count] } result table: 4, state tables: [] - StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr199, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr200] } + StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr213, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr214] } Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } Upstream BatchPlanNode Fragment 2 - StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr197, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr198, auction.id] } + StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr215, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr216, auction.id] } Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } Upstream BatchPlanNode - Table 0 { columns: [person_id, person_name, $expr199, $expr200], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } - Table 1 { columns: [person_id, $expr199, $expr200, person_name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } - Table 2 { columns: [auction_seller, $expr197, $expr198], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } - Table 3 { columns: [auction_seller, $expr197, $expr198, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4 { columns: [person_id, person_name, $expr199, $expr200, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } - Table 5 { columns: [auction_seller, $expr197, $expr198, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4294967294 { columns: [id, name, starttime, $expr200, auction.seller, $expr197, $expr198], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } + Table 0 { columns: [person_id, person_name, $expr213, $expr214], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } + Table 1 { columns: [person_id, $expr213, $expr214, person_name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } + Table 2 { columns: [auction_seller, $expr215, $expr216], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [auction_seller, $expr215, $expr216, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [person_id, person_name, $expr213, $expr214, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } + Table 5 { columns: [auction_seller, $expr215, $expr216, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4294967294 { columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } - id: nexmark_q9 before: - create_tables @@ -588,7 +588,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id], distribution: UpstreamHashShard(bid._row_id) } stream_plan: | - StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id(hidden)], pk_columns: [id, bid._row_id, auction] } + StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id(hidden)], pk_columns: [id, bid._row_id, auction], pk_conflict: "no check" } └─StreamProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id] } └─StreamGroupTopN { order: "[bid.price DESC, bid.date_time ASC]", limit: 1, offset: 0, group_key: [0] } └─StreamFilter { predicate: (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } @@ -599,7 +599,7 @@ └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id(hidden)], pk_columns: [id, bid._row_id, auction] } + StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id(hidden)], pk_columns: [id, bid._row_id, auction], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id] } StreamGroupTopN { order: "[bid.price DESC, bid.date_time ASC]", limit: 1, offset: 0, group_key: [0] } @@ -636,14 +636,14 @@ └─BatchProject { 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] } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } └─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] } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr95, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr96, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr103, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr104, 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) } Upstream BatchPlanNode @@ -724,15 +724,15 @@ └─BatchFilter { predicate: ((0.908:Decimal * bid.price) > 1000000:Int32) AND ((0.908:Decimal * bid.price) < 50000000:Int32) } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid.extra], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr1, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, bid.date_time, bid.extra, bid._row_id] } └─StreamFilter { predicate: ((0.908:Decimal * bid.price) > 1000000:Int32) AND ((0.908:Decimal * bid.price) < 50000000:Int32) } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr95, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr96, bid.date_time, bid.extra, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr103, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr104, bid.date_time, bid.extra, bid._row_id] } StreamFilter { predicate: ((0.908:Decimal * bid.price) > 1000000:Int32) AND ((0.908:Decimal * bid.price) < 50000000:Int32) } 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) } Upstream @@ -770,28 +770,36 @@ └─BatchProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, bid.price, bid.bidder, bid.auction] } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day] } - └─StreamProject { exprs: [$expr1, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } + └─StreamProject { exprs: [$expr1, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [$expr1], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, bid.price, bid.bidder, bid.auction, 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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day] } + StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [$expr48, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [$expr48], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + StreamProject { exprs: [$expr52, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamHashAgg { group_key: [$expr52], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr48, 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) } - Upstream - BatchPlanNode + StreamProject { exprs: [$expr52, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [$expr52, bid.bidder, bid.auction, flag], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + result table: 1, state tables: [] + StreamExchange Hash([0, 2, 3, 10]) from 2 + + Fragment 2 + StreamExpand { column_subsets: [[$expr52], [$expr52, bid.bidder], [$expr52, bid.auction]] } + StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr52, 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) } + Upstream + BatchPlanNode - Table 0 { columns: [$expr48, count, count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } + Table 0 { columns: [$expr52, count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder) filter((flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction) filter((flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } + Table 1 { columns: [$expr52, bid_bidder, bid_auction, flag, count, count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count filter((bid_price < 10000:Int32))_0, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_0, count filter((bid_price >= 1000000:Int32))_0, count filter((bid_price < 10000:Int32))_1, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_1, count filter((bid_price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1, 2, 3] } Table 4294967294 { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } - id: nexmark_q16 before: @@ -826,28 +834,37 @@ └─BatchProject { 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] } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day] } - └─StreamProject { exprs: [bid.channel, $expr1, max($expr2), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr1], aggs: [count, max($expr2), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } + └─StreamProject { exprs: [bid.channel, $expr1, max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [bid.channel, $expr1], aggs: [count, max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } └─StreamExchange { dist: HashShard(bid.channel, $expr1) } └─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] } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day] } + StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.channel, $expr95, max($expr96), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr95], aggs: [count, max($expr96), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - result table: 0, state tables: [] + StreamProject { exprs: [bid.channel, $expr103, max(max($expr104)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamHashAgg { group_key: [bid.channel, $expr103], aggs: [count, max(max($expr104)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + result table: 1, state tables: [0] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr95, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr96, 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) } - Upstream - BatchPlanNode + StreamProject { exprs: [bid.channel, $expr103, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr104), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr103, bid.bidder, bid.auction, flag], aggs: [count, max($expr104), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + result table: 2, state tables: [] + StreamExchange Hash([0, 1, 4, 5, 14]) from 2 + + Fragment 2 + StreamExpand { column_subsets: [[bid.channel, $expr103, $expr104], [bid.channel, $expr103, bid.bidder], [bid.channel, $expr103, bid.auction]] } + StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr103, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr104, 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) } + Upstream + BatchPlanNode - Table 0 { columns: [bid_channel, $expr95, count, max($expr96), count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0, 1] } + Table 0 { columns: [bid_channel, $expr103, max($expr104), bid_bidder, bid_auction, flag], primary key: [$0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0, 1] } + Table 1 { columns: [bid_channel, $expr103, count, max(max($expr104)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder) filter((flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction) filter((flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0, 1] } + Table 2 { columns: [bid_channel, $expr103, bid_bidder, bid_auction, flag, count, max($expr104), count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count filter((bid_price < 10000:Int32))_0, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_0, count filter((bid_price >= 1000000:Int32))_0, count filter((bid_price < 10000:Int32))_1, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_1, count filter((bid_price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [0, 1, 2, 3, 4] } Table 4294967294 { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } - id: nexmark_q17 before: @@ -874,7 +891,7 @@ └─BatchProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr1, bid.price] } └─BatchScan { table: bid, columns: [bid.auction, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day] } + StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } └─StreamProject { exprs: [bid.auction, $expr1, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr2, sum(bid.price)] } └─StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr1], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price), sum(bid.price)] } └─StreamExchange { dist: HashShard(bid.auction, $expr1) } @@ -882,20 +899,20 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day] } + StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, $expr95, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr97, sum(bid.price)] } - StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr95], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price), sum(bid.price)] } + StreamProject { exprs: [bid.auction, $expr104, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr105, sum(bid.price)] } + StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr104], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price), sum(bid.price)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr95, bid.price, bid._row_id] } + StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr104, 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) } Upstream BatchPlanNode - Table 0 { columns: [bid_auction, $expr95, count, count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price), sum(bid_price)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [0, 1] } + Table 0 { columns: [bid_auction, $expr104, count, count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price), sum(bid_price)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [0, 1] } Table 4294967294 { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: nexmark_q18 before: @@ -918,7 +935,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.bidder, bid.auction) } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id], distribution: UpstreamHashShard(bid._row_id) } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(bid._row_id) } └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id] } └─StreamAppendOnlyGroupTopN { order: "[bid.date_time DESC]", limit: 1, offset: 0, group_key: [1, 0] } @@ -926,7 +943,7 @@ └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([7]) from 1 @@ -978,7 +995,7 @@ └─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: | - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], pk_columns: [bid._row_id, auction.id, auction] } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], pk_columns: [bid._row_id, auction.id, auction], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, 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, bid._row_id, auction.id] } ├─StreamExchange { dist: HashShard(bid.auction) } | └─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) } @@ -987,7 +1004,7 @@ └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], pk_columns: [bid._row_id, auction.id, auction] } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], pk_columns: [bid._row_id, auction.id, auction], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, 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, bid._row_id, auction.id] } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -1043,14 +1060,14 @@ └─BatchProject { 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] } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } └─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] } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr142, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr143, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr144, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr154, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr155, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr156, 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) } Upstream BatchPlanNode @@ -1085,7 +1102,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction, bid.price], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], pk_columns: [auction_id, bid.auction] } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], pk_columns: [auction_id, bid.auction], pk_conflict: "no check" } └─StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } ├─StreamExchange { dist: HashShard(auction.id) } | └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -1095,7 +1112,7 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], pk_columns: [auction_id, bid.auction] } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], pk_columns: [auction_id, bid.auction], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -1155,7 +1172,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name] } + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } ├─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } @@ -1172,10 +1189,10 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name] } + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } - StreamDynamicFilter { predicate: (count(bid.auction) >= $expr49), output: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } + StreamDynamicFilter { predicate: (count(bid.auction) >= $expr53), output: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } left table: 0, right table 1 StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction), count(bid.auction)] } @@ -1197,8 +1214,8 @@ BatchPlanNode Fragment 3 - StreamProject { exprs: [(count / count(distinct bid.auction)) as $expr49] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count, count, count(distinct bid.auction)] } + StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr53] } + StreamGlobalSimpleAgg { aggs: [count, sum0(count), count(bid.auction)] } result table: 7, state tables: [] StreamExchange Single from 4 @@ -1208,7 +1225,7 @@ BatchPlanNode Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), count(bid_auction)_0], primary key: [$3 ASC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } - Table 1 { columns: [$expr49], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr53], primary key: [], value indices: [0], distribution key: [] } Table 2 { columns: [auction_id, auction_item_name, count, count(bid_auction), count(bid_auction)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4], distribution key: [0] } Table 3 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } @@ -1243,7 +1260,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction_id, auction_item_name], pk_columns: [auction_id] } + StreamMaterialize { columns: [auction_id, auction_item_name], pk_columns: [auction_id], pk_conflict: "no check" } └─StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all } ├─StreamExchange { dist: HashShard(auction.id) } | └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -1255,7 +1272,7 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name], pk_columns: [auction_id] } + StreamMaterialize { columns: [auction_id, auction_item_name], pk_columns: [auction_id], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -1310,7 +1327,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction_id, auction_item_name], pk_columns: [auction_id] } + StreamMaterialize { columns: [auction_id, auction_item_name], pk_columns: [auction_id], pk_conflict: "no check" } └─StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all } ├─StreamExchange { dist: HashShard(auction.id) } | └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -1322,7 +1339,7 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name], pk_columns: [auction_id] } + StreamMaterialize { columns: [auction_id, auction_item_name], pk_columns: [auction_id], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -1377,7 +1394,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], order_descs: [bid_count, auction_id, auction_item_name] } + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], order_descs: [bid_count, auction_id, auction_item_name], pk_conflict: "no check" } └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } └─StreamTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0 } └─StreamExchange { dist: Single } @@ -1392,7 +1409,7 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], order_descs: [bid_count, auction_id, auction_item_name] } + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], order_descs: [bid_count, auction_id, auction_item_name], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } StreamTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0 } @@ -1402,7 +1419,7 @@ Fragment 1 StreamGroupTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0, group_key: [3] } state table: 1 - StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr1] } + StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr3] } StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction)] } result table: 2, state tables: [] @@ -1421,8 +1438,8 @@ Upstream BatchPlanNode - Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), $expr1], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [auction_id, auction_item_name, count(bid_auction), $expr1], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } + Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), $expr3], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } + Table 1 { columns: [auction_id, auction_item_name, count(bid_auction), $expr3], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } Table 2 { columns: [auction_id, auction_item_name, count, count(bid_auction)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index 55181791a5b1e..cc23a0961b2fd 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -51,14 +51,14 @@ └─BatchProject { exprs: [auction, bidder, price, date_time] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([4]) from 1 @@ -85,19 +85,19 @@ └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr1, date_time] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr1, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([4]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr50, date_time, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr54, date_time, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -114,7 +114,7 @@ └─BatchFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, price, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, price, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamProject { exprs: [auction, price, _row_id] } └─StreamFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) } @@ -122,7 +122,7 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, price, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([2]) from 1 @@ -157,7 +157,7 @@ └─BatchFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } └─BatchSource { source: "person", columns: ["id", "name", "email_address", "credit_card", "city", "state", "date_time", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden), id#1(hidden)], pk_columns: [_row_id, _row_id#1, seller, id#1] } + StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden), id#1(hidden)], pk_columns: [_row_id, _row_id#1, seller, id#1], pk_conflict: "no check" } └─StreamAppendOnlyHashJoin { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id, id] } ├─StreamExchange { dist: HashShard(seller) } | └─StreamProject { exprs: [id, seller, _row_id] } @@ -171,7 +171,7 @@ └─StreamSource { source: "person", columns: ["id", "name", "email_address", "credit_card", "city", "state", "date_time", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden), id#1(hidden)], pk_columns: [_row_id, _row_id#1, seller, id#1] } + StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden), id#1(hidden)], pk_columns: [_row_id, _row_id#1, seller, id#1], pk_conflict: "no check" } materialized table: 4294967294 StreamAppendOnlyHashJoin { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id, id] } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -229,7 +229,7 @@ └─BatchProject { exprs: [auction, price, date_time] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [category, avg], pk_columns: [category] } + StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } └─StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr1] } └─StreamHashAgg { group_key: [category], aggs: [count, sum(max(price)), count(max(price))] } └─StreamExchange { dist: HashShard(category) } @@ -248,9 +248,9 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [category, avg], pk_columns: [category] } + StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr49] } + StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr53] } StreamHashAgg { group_key: [category], aggs: [count, sum(max(price)), count(max(price))] } result table: 0, state tables: [] StreamExchange Hash([1]) from 1 @@ -340,7 +340,7 @@ └─BatchProject { exprs: [auction, date_time] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [window_start, auction, window_start#1] } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [window_start, auction, window_start#1], pk_conflict: "no check" } └─StreamProject { exprs: [auction, count, window_start, window_start] } └─StreamFilter { predicate: (count >= max(count)) } └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } @@ -350,7 +350,7 @@ | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamShare { id = 854 } + | └─StreamShare { id = 1066 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -362,13 +362,13 @@ └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamShare { id = 854 } + └─StreamShare { id = 1066 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [window_start, auction, window_start#1] } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [window_start, auction, window_start#1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [auction, count, window_start, window_start] } StreamFilter { predicate: (count >= max(count)) } @@ -474,13 +474,13 @@ └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr112(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr112, price, max(price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr121(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr121, price, max(price)], pk_conflict: "no check" } └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1, max(price)] } └─StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } ├─StreamExchange { dist: HashShard(price) } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } - | └─StreamShare { id = 503 } + | └─StreamShare { id = 641 } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -489,16 +489,16 @@ └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, max(price)] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } - └─StreamShare { id = 503 } + └─StreamShare { id = 641 } └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr112(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr112, price, max(price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr121(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr121, price, max(price)], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr112, max(price)] } - StreamFilter { predicate: (date_time >= $expr114) AND (date_time <= $expr112) } + StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr121, max(price)] } + StreamFilter { predicate: (date_time >= $expr122) AND (date_time <= $expr121) } StreamHashJoin { type: Inner, predicate: price = max(price), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([2]) from 1 @@ -515,22 +515,22 @@ source state table: 4 Fragment 3 - StreamProject { exprs: [max(price), $expr112, ($expr112 - '00:00:10':Interval) as $expr114] } - StreamAppendOnlyHashAgg { group_key: [$expr112], aggs: [count, max(price)] } + StreamProject { exprs: [max(price), $expr121, ($expr121 - '00:00:10':Interval) as $expr122] } + StreamAppendOnlyHashAgg { group_key: [$expr121], aggs: [count, max(price)] } result table: 5, state tables: [] StreamExchange Hash([0]) from 4 Fragment 4 - StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr112, price, _row_id] } + StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr121, price, _row_id] } StreamExchange Hash([4]) from 2 Table 0 { columns: [auction, bidder, price, date_time, _row_id], primary key: [$2 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } Table 1 { columns: [price, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 2 { columns: [max(price), $expr112, $expr114], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 3 { columns: [max(price), $expr112, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 2 { columns: [max(price), $expr121, $expr122], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(price), $expr121, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 5 { columns: [$expr112, count, max(price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr112, max(price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 5 { columns: [$expr121, count, max(price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr121, max(price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - id: nexmark_q8 before: - create_sources @@ -580,7 +580,7 @@ └─BatchProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr3, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4] } └─BatchSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [id, name, starttime, $expr208(hidden), seller(hidden), $expr205(hidden), $expr206(hidden)], pk_columns: [id, name, starttime, $expr208, seller, $expr205, $expr206] } + StreamMaterialize { columns: [id, name, starttime, $expr222(hidden), seller(hidden), $expr223(hidden), $expr224(hidden)], pk_columns: [id, name, starttime, $expr222, seller, $expr223, $expr224], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) } | └─StreamProject { exprs: [id, name, $expr1, $expr2] } @@ -597,43 +597,43 @@ └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr208(hidden), seller(hidden), $expr205(hidden), $expr206(hidden)], pk_columns: [id, name, starttime, $expr208, seller, $expr205, $expr206] } + StreamMaterialize { columns: [id, name, starttime, $expr222(hidden), seller(hidden), $expr223(hidden), $expr224(hidden)], pk_columns: [id, name, starttime, $expr222, seller, $expr223, $expr224], pk_conflict: "no check" } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: id = seller AND $expr207 = $expr205 AND $expr208 = $expr206, output: all } + StreamHashJoin { type: Inner, predicate: id = seller AND $expr221 = $expr223 AND $expr222 = $expr224, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0, 2, 3]) from 1 - StreamProject { exprs: [seller, $expr205, $expr206] } - StreamAppendOnlyHashAgg { group_key: [seller, $expr205, $expr206], aggs: [count] } + StreamProject { exprs: [seller, $expr223, $expr224] } + StreamAppendOnlyHashAgg { group_key: [seller, $expr223, $expr224], aggs: [count] } result table: 6, state tables: [] StreamExchange Hash([0, 1, 2]) from 3 Fragment 1 - StreamProject { exprs: [id, name, $expr207, $expr208] } - StreamAppendOnlyHashAgg { group_key: [id, name, $expr207, $expr208], aggs: [count] } + StreamProject { exprs: [id, name, $expr221, $expr222] } + StreamAppendOnlyHashAgg { group_key: [id, name, $expr221, $expr222], aggs: [count] } result table: 4, state tables: [] StreamExchange Hash([0, 1, 2, 3]) from 2 Fragment 2 - StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr207, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr208, _row_id] } + StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr221, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr222, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "person", columns: ["id", "name", "email_address", "credit_card", "city", "state", "date_time", "_row_id"] } source state table: 5 Fragment 3 - StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr205, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr206, _row_id] } + StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr223, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr224, _row_id] } StreamRowIdGen { row_id_index: 9 } StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } source state table: 7 - Table 0 { columns: [id, name, $expr207, $expr208], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } - Table 1 { columns: [id, $expr207, $expr208, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } - Table 2 { columns: [seller, $expr205, $expr206], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } - Table 3 { columns: [seller, $expr205, $expr206, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4 { columns: [id, name, $expr207, $expr208, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2, 3] } + Table 0 { columns: [id, name, $expr221, $expr222], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } + Table 1 { columns: [id, $expr221, $expr222, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } + Table 2 { columns: [seller, $expr223, $expr224], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [seller, $expr223, $expr224, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [id, name, $expr221, $expr222, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2, 3] } Table 5 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 6 { columns: [seller, $expr205, $expr206, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 6 { columns: [seller, $expr223, $expr224, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } Table 7 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 4294967294 { columns: [id, name, starttime, $expr208, seller, $expr205, $expr206], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } + Table 4294967294 { columns: [id, name, starttime, $expr222, seller, $expr223, $expr224], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } - id: nexmark_q9 before: - create_sources @@ -674,7 +674,7 @@ └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, id, auction] } + StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, id, auction], pk_conflict: "no check" } └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time, _row_id, _row_id] } └─StreamAppendOnlyGroupTopN { order: "[price DESC, date_time ASC]", limit: 1, offset: 0, group_key: [0] } └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } @@ -687,7 +687,7 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, id, auction] } + StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, id, auction], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time, _row_id, _row_id] } StreamAppendOnlyGroupTopN { order: "[price DESC, date_time ASC]", limit: 1, offset: 0, group_key: [0] } @@ -726,19 +726,19 @@ └─BatchProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(date_time, 'HH:MI':Varchar) as $expr2] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(date_time, 'HH:MI':Varchar) as $expr2, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([6]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr99, ToChar(date_time, 'HH:MI':Varchar) as $expr100, _row_id] } + StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr107, ToChar(date_time, 'HH:MI':Varchar) as $expr108, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -820,7 +820,7 @@ └─BatchFilter { predicate: ((0.908:Decimal * price) > 1000000:Int32) AND ((0.908:Decimal * price) < 50000000:Int32) } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra, _row_id] } └─StreamFilter { predicate: ((0.908:Decimal * price) > 1000000:Int32) AND ((0.908:Decimal * price) < 50000000:Int32) } @@ -828,12 +828,12 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([6]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr99, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr100, date_time, extra, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr107, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr108, date_time, extra, _row_id] } StreamFilter { predicate: ((0.908:Decimal * price) > 1000000:Int32) AND ((0.908:Decimal * price) < 50000000:Int32) } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -872,30 +872,38 @@ └─BatchProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day] } - └─StreamProject { exprs: [$expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } + └─StreamProject { exprs: [$expr1, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [$expr1], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day] } + StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [$expr50, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [$expr50], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + StreamProject { exprs: [$expr54, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamHashAgg { group_key: [$expr54], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr50, price, bidder, auction, _row_id] } - StreamRowIdGen { row_id_index: 7 } - StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } - source state table: 3 + StreamProject { exprs: [$expr54, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [$expr54, bidder, auction, flag], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + result table: 1, state tables: [] + StreamExchange Hash([0, 2, 3, 10]) from 2 - Table 0 { columns: [$expr50, count, count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } - Table 3 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } + Fragment 2 + StreamExpand { column_subsets: [[$expr54], [$expr54, bidder], [$expr54, auction]] } + StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr54, price, bidder, auction, _row_id] } + StreamRowIdGen { row_id_index: 7 } + StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + source state table: 2 + + Table 0 { columns: [$expr54, count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } + Table 1 { columns: [$expr54, bidder, auction, flag, count, count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32))_0, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_0, count filter((price >= 1000000:Int32))_0, count filter((price < 10000:Int32))_1, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_1, count filter((price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1, 2, 3] } + Table 2 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } - id: nexmark_q16 before: @@ -930,29 +938,38 @@ └─BatchProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day] } - └─StreamProject { exprs: [channel, $expr1, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [channel, $expr1], aggs: [count, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } + └─StreamProject { exprs: [channel, $expr1, max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [channel, $expr1], aggs: [count, max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } └─StreamExchange { dist: HashShard(channel, $expr1) } └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day] } + StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [channel, $expr99, max($expr100), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [channel, $expr99], aggs: [count, max($expr100), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - result table: 0, state tables: [] + StreamProject { exprs: [channel, $expr107, max(max($expr108)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamHashAgg { group_key: [channel, $expr107], aggs: [count, max(max($expr108)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + result table: 1, state tables: [0] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr99, ToChar(date_time, 'HH:mm':Varchar) as $expr100, price, bidder, auction, _row_id] } - StreamRowIdGen { row_id_index: 7 } - StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } - source state table: 3 + StreamProject { exprs: [channel, $expr107, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr108), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [channel, $expr107, bidder, auction, flag], aggs: [count, max($expr108), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + result table: 2, state tables: [] + StreamExchange Hash([0, 1, 4, 5, 14]) from 2 + + Fragment 2 + StreamExpand { column_subsets: [[channel, $expr107, $expr108], [channel, $expr107, bidder], [channel, $expr107, auction]] } + StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr107, ToChar(date_time, 'HH:mm':Varchar) as $expr108, price, bidder, auction, _row_id] } + StreamRowIdGen { row_id_index: 7 } + StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + source state table: 3 - Table 0 { columns: [channel, $expr99, count, max($expr100), count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0, 1] } + Table 0 { columns: [channel, $expr107, max($expr108), bidder, auction, flag], primary key: [$0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0, 1] } + Table 1 { columns: [channel, $expr107, count, max(max($expr108)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0, 1] } + Table 2 { columns: [channel, $expr107, bidder, auction, flag, count, max($expr108), count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32))_0, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_0, count filter((price >= 1000000:Int32))_0, count filter((price < 10000:Int32))_1, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_1, count filter((price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [0, 1, 2, 3, 4] } Table 3 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } - id: nexmark_q17 @@ -980,7 +997,7 @@ └─BatchProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, price] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day] } + StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } └─StreamProject { exprs: [auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr2, sum(price)] } └─StreamAppendOnlyHashAgg { group_key: [auction, $expr1], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)] } └─StreamExchange { dist: HashShard(auction, $expr1) } @@ -989,20 +1006,20 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day] } + StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction, $expr97, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr99, sum(price)] } - StreamAppendOnlyHashAgg { group_key: [auction, $expr97], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)] } + StreamProject { exprs: [auction, $expr106, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr107, sum(price)] } + StreamAppendOnlyHashAgg { group_key: [auction, $expr106], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr97, price, _row_id] } + StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr106, price, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 1 - Table 0 { columns: [auction, $expr97, count, count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [0, 1] } + Table 0 { columns: [auction, $expr106, count, count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [0, 1] } Table 1 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: nexmark_q18 @@ -1026,7 +1043,7 @@ └─BatchExchange { order: [], dist: HashShard(bidder, auction) } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─StreamAppendOnlyGroupTopN { order: "[date_time DESC]", limit: 1, offset: 0, group_key: [1, 0] } @@ -1035,7 +1052,7 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([7]) from 1 @@ -1093,7 +1110,7 @@ └─BatchFilter { predicate: (category = 10:Int32) } └─BatchSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden), id(hidden)], pk_columns: [_row_id, _row_id#1, auction, id] } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden), id(hidden)], pk_columns: [_row_id, _row_id#1, auction, id], pk_conflict: "no check" } └─StreamAppendOnlyHashJoin { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id, id] } ├─StreamExchange { dist: HashShard(auction) } | └─StreamProject { exprs: [auction, bidder, price, channel, url, date_time, _row_id] } @@ -1106,7 +1123,7 @@ └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden), id(hidden)], pk_columns: [_row_id, _row_id#1, auction, id] } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden), id(hidden)], pk_columns: [_row_id, _row_id#1, auction, id], pk_conflict: "no check" } materialized table: 4294967294 StreamAppendOnlyHashJoin { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id, id] } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -1166,19 +1183,19 @@ └─BatchProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr1, SplitPart(url, '/':Varchar, 5:Int32) as $expr2, SplitPart(url, '/':Varchar, 6:Int32) as $expr3] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr1, SplitPart(url, '/':Varchar, 5:Int32) as $expr2, SplitPart(url, '/':Varchar, 6:Int32) as $expr3, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([7]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr148, SplitPart(url, '/':Varchar, 5:Int32) as $expr149, SplitPart(url, '/':Varchar, 6:Int32) as $expr150, _row_id] } + StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr160, SplitPart(url, '/':Varchar, 5:Int32) as $expr161, SplitPart(url, '/':Varchar, 6:Int32) as $expr162, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -1215,7 +1232,7 @@ └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], pk_columns: [_row_id, auction, auction_id] } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], pk_columns: [_row_id, auction, auction_id], pk_conflict: "no check" } └─StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, item_name, _row_id] } @@ -1228,7 +1245,7 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], pk_columns: [_row_id, auction, auction_id] } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], pk_columns: [_row_id, auction, auction_id], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -1294,7 +1311,7 @@ └─BatchProject { exprs: [auction] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name] } + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } └─StreamProject { exprs: [id, item_name, count(auction)] } └─StreamDynamicFilter { predicate: (count(auction) >= $expr1), output: [id, item_name, count(auction), count(auction)] } ├─StreamProject { exprs: [id, item_name, count(auction), count(auction)] } @@ -1306,7 +1323,7 @@ | | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } | └─StreamExchange { dist: HashShard(auction) } | └─StreamProject { exprs: [auction, _row_id] } - | └─StreamShare { id = 634 } + | └─StreamShare { id = 842 } | └─StreamProject { exprs: [auction, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1314,17 +1331,20 @@ └─StreamProject { exprs: [(count / count(distinct auction)) as $expr1] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, count, count(distinct auction)] } └─StreamExchange { dist: Single } - └─StreamProject { exprs: [auction, _row_id] } - └─StreamShare { id = 634 } - └─StreamProject { exprs: [auction, _row_id] } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + └─StreamProject { exprs: [auction, count] } + └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamProject { exprs: [auction, _row_id] } + └─StreamShare { id = 842 } + └─StreamProject { exprs: [auction, _row_id] } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name] } + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [id, item_name, count(auction)] } - StreamDynamicFilter { predicate: (count(auction) >= $expr51), output: [id, item_name, count(auction), count(auction)] } + StreamDynamicFilter { predicate: (count(auction) >= $expr55), output: [id, item_name, count(auction), count(auction)] } left table: 0, right table 1 StreamProject { exprs: [id, item_name, count(auction), count(auction)] } StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction), count(auction)] } @@ -1352,8 +1372,8 @@ source state table: 8 Fragment 4 - StreamProject { exprs: [(count / count(distinct auction)) as $expr51] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count, count, count(distinct auction)] } + StreamProject { exprs: [(sum0(count) / count(auction)) as $expr55] } + StreamGlobalSimpleAgg { aggs: [count, sum0(count), count(auction)] } result table: 9, state tables: [] StreamExchange Single from 5 @@ -1362,7 +1382,7 @@ StreamExchange Hash([1]) from 3 Table 0 { columns: [id, item_name, count(auction), count(auction)_0], primary key: [$3 ASC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } - Table 1 { columns: [$expr51], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr55], primary key: [], value indices: [0], distribution key: [] } Table 2 { columns: [id, item_name, count, count(auction), count(auction)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4], distribution key: [0] } Table 3 { columns: [id, item_name, _row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 4 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } @@ -1400,7 +1420,7 @@ └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], pk_columns: [_row_id, auction_id] } + StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], pk_columns: [_row_id, auction_id], pk_conflict: "no check" } └─StreamHashJoin { type: LeftSemi, predicate: id = auction, output: all } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, item_name, _row_id] } @@ -1415,7 +1435,7 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], pk_columns: [_row_id, auction_id] } + StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], pk_columns: [_row_id, auction_id], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: LeftSemi, predicate: id = auction, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -1474,7 +1494,7 @@ └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], pk_columns: [_row_id, auction_id] } + StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], pk_columns: [_row_id, auction_id], pk_conflict: "no check" } └─StreamHashJoin { type: LeftAnti, predicate: id = auction, output: all } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, item_name, _row_id] } @@ -1489,7 +1509,7 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], pk_columns: [_row_id, auction_id] } + StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], pk_columns: [_row_id, auction_id], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: LeftAnti, predicate: id = auction, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -1549,7 +1569,7 @@ └─BatchProject { exprs: [auction] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], order_descs: [bid_count, auction_id, auction_item_name] } + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], order_descs: [bid_count, auction_id, auction_item_name], pk_conflict: "no check" } └─StreamProject { exprs: [id, item_name, count(auction)] } └─StreamTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0 } └─StreamExchange { dist: Single } @@ -1568,7 +1588,7 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], order_descs: [bid_count, auction_id, auction_item_name] } + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], order_descs: [bid_count, auction_id, auction_item_name], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [id, item_name, count(auction)] } StreamTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0 } @@ -1578,7 +1598,7 @@ Fragment 1 StreamGroupTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0, group_key: [3] } state table: 1 - StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr3] } StreamProject { exprs: [id, item_name, count(auction)] } StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction)] } result table: 2, state tables: [] @@ -1599,8 +1619,8 @@ StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 8 - Table 0 { columns: [id, item_name, count(auction), $expr1], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [id, item_name, count(auction), $expr1], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } + Table 0 { columns: [id, item_name, count(auction), $expr3], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } + Table 1 { columns: [id, item_name, count(auction), $expr3], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } Table 2 { columns: [id, item_name, count, count(auction)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [id, item_name, _row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 4 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index 2ada1226dce34..e8f42e306cac6 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -8,7 +8,7 @@ └─BatchSort { order: [t.v1 DESC] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [v1, t._row_id] } + StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [v1, t._row_id], pk_conflict: "no check" } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: output names are not qualified after table names sql: | @@ -59,13 +59,13 @@ select * from t order by 1+1; batch_plan: | BatchProject { exprs: [t.v1, t.v2] } - └─BatchExchange { order: [$expr1 ASC], dist: Single } - └─BatchSort { order: [$expr1 ASC] } - └─BatchProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32) as $expr1] } + └─BatchExchange { order: [2:Int32 ASC], dist: Single } + └─BatchSort { order: [2:Int32 ASC] } + └─BatchProject { exprs: [t.v1, t.v2, 2:Int32] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, $expr48(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr48, t._row_id] } - └─StreamProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32) as $expr1, t._row_id] } + StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [2:Int32, t._row_id], pk_conflict: "no check" } + └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 bigint, v2 double precision); @@ -80,7 +80,7 @@ └─BatchTopN { order: "[t.v1 DESC]", limit: 5, offset: 0 } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [v1, t._row_id] } + StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [v1, t._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t.v1, t.v2, t._row_id] } └─StreamTopN { order: "[t.v1 DESC]", limit: 5, offset: 0 } └─StreamExchange { dist: Single } @@ -112,7 +112,7 @@ └─BatchTopN { order: "[t.v1 DESC]", limit: 12, offset: 0 } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [v1, t._row_id] } + StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [v1, t._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t.v1, t.v2, t._row_id] } └─StreamTopN { order: "[t.v1 DESC]", limit: 5, offset: 7 } └─StreamExchange { dist: Single } @@ -133,7 +133,7 @@ └─BatchProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z] } └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [x, y, $expr67(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr67, t.z, t._row_id] } + StreamMaterialize { columns: [x, y, $expr71(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr71, t.z, t._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z, t._row_id] } └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by the number of an output column @@ -214,3 +214,24 @@ └─BatchSort { order: [$expr1 ASC] } └─BatchProject { exprs: [(test.b % 2:Int32) as $expr1, test.a] } └─BatchScan { table: test, columns: [test.a, test.b], distribution: SomeShard } +- name: Orderby with always-false predicate + sql: | + create table t1 (v1 int); + select + 1 as col_0 + from + t1 as t_0 + where + false + group by + t_0.v1 + order by + t_0.v1 asc; + batch_plan: | + BatchProject { exprs: [1:Int32] } + └─BatchExchange { order: [t1.v1 ASC], dist: Single } + └─BatchProject { exprs: [1:Int32, t1.v1] } + └─BatchSortAgg { group_key: [t1.v1], aggs: [] } + └─BatchExchange { order: [t1.v1 ASC], dist: HashShard(t1.v1) } + └─BatchSort { order: [t1.v1 ASC] } + └─BatchValues { rows: [] } diff --git a/src/frontend/planner_test/tests/testdata/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/over_window_function.yaml index be4bf3c5f4635..92f73a3f043fc 100644 --- a/src/frontend/planner_test/tests/testdata/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/over_window_function.yaml @@ -104,7 +104,7 @@ └─BatchFilter { predicate: (t.x > t.y) } └─BatchScan { table: t, columns: [t.x, t.y, t._row_id], distribution: UpstreamHashShard(t._row_id) } stream_plan: | - StreamMaterialize { columns: [x, y, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [x, y, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(t._row_id) } └─StreamProject { exprs: [t.x, t.y, t._row_id] } └─StreamGroupTopN { order: "[t.x ASC]", limit: 2, offset: 0, group_key: [1] } @@ -132,7 +132,7 @@ └─BatchExchange { order: [], dist: HashShard(t.y) } └─BatchScan { table: t, columns: [t.x, t.y, t._row_id], distribution: UpstreamHashShard(t._row_id) } stream_plan: | - StreamMaterialize { columns: [x, y, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [x, y, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(t._row_id) } └─StreamProject { exprs: [t.x, t.y, t._row_id] } └─StreamGroupTopN { order: "[t.x ASC]", limit: 3, offset: 0, group_key: [1], with_ties: true } @@ -288,7 +288,7 @@ ) ) WHERE rownum <= 3; stream_plan: | - StreamMaterialize { columns: [window_start, window_end, supplier_id, price, cnt], pk_columns: [window_start, window_end, supplier_id] } + StreamMaterialize { columns: [window_start, window_end, supplier_id, price, cnt], pk_columns: [window_start, window_end, supplier_id], pk_conflict: "no check" } └─StreamGroupTopN { order: "[sum(bid.price) DESC]", limit: 3, offset: 0, group_key: [0, 1] } └─StreamExchange { dist: HashShard($expr1, $expr2) } └─StreamProject { exprs: [$expr1, $expr2, bid.supplier_id, sum(bid.price), count] } @@ -308,7 +308,7 @@ TUMBLE(Bid, bidtime, INTERVAL '10' MINUTE) ) WHERE rownum <= 3; stream_plan: | - StreamMaterialize { columns: [window_start, window_end, supplier_id, price, bid._row_id(hidden)], pk_columns: [bid._row_id] } + StreamMaterialize { columns: [window_start, window_end, supplier_id, price, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(bid._row_id) } └─StreamProject { exprs: [$expr1, $expr2, bid.supplier_id, bid.price, bid._row_id] } └─StreamGroupTopN { order: "[bid.price DESC]", limit: 3, offset: 0, group_key: [5, 6] } @@ -332,7 +332,7 @@ └─LogicalTopN { order: "[t.y ASC]", limit: 1, offset: 0, group_key: [0] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } stream_plan: | - StreamMaterialize { columns: [x, y, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [x, y, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(t._row_id) } └─StreamProject { exprs: [t.x, t.y, t._row_id] } └─StreamGroupTopN { order: "[t.y ASC]", limit: 1, offset: 0, group_key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/pk_derive.yaml b/src/frontend/planner_test/tests/testdata/pk_derive.yaml index 3ec7e4f7c06c8..4fc006614482c 100644 --- a/src/frontend/planner_test/tests/testdata/pk_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/pk_derive.yaml @@ -20,7 +20,7 @@ ON Tone.id = Ttwo.id; stream_plan: | - StreamMaterialize { columns: [max_v1, max_v2, t1.id(hidden), t2.id(hidden)], pk_columns: [t1.id, t2.id] } + StreamMaterialize { columns: [max_v1, max_v2, t1.id(hidden), t2.id(hidden)], pk_columns: [t1.id, t2.id], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.id = t2.id, output: [max(t1.v1), max(t2.v2), t1.id, t2.id] } ├─StreamProject { exprs: [max(t1.v1), t1.id] } | └─StreamHashAgg { group_key: [t1.id], aggs: [count, max(t1.v1)] } @@ -50,7 +50,7 @@ ON Tone.id = Ttwo.id; stream_plan: | - StreamMaterialize { columns: [max_v, min_v, t.id(hidden), t.id#1(hidden)], pk_columns: [t.id, t.id#1] } + StreamMaterialize { columns: [max_v, min_v, t.id(hidden), t.id#1(hidden)], pk_columns: [t.id, t.id#1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t.id = t.id, output: [max(t.v), min(t.v), t.id, t.id] } ├─StreamProject { exprs: [max(t.v), t.id] } | └─StreamHashAgg { group_key: [t.id], aggs: [count, max(t.v)] } @@ -74,7 +74,7 @@ LogicalAgg { group_key: [t.v1, t.v2, t.v3], aggs: [] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3] } stream_plan: | - StreamMaterialize { columns: [v1, v2, v3], pk_columns: [v1, v2, v3] } + StreamMaterialize { columns: [v1, v2, v3], pk_columns: [v1, v2, v3], pk_conflict: "no check" } └─StreamProject { exprs: [t.v1, t.v2, t.v3] } └─StreamHashAgg { group_key: [t.v1, t.v2, t.v3], aggs: [count] } └─StreamExchange { dist: HashShard(t.v1, t.v2, t.v3) } @@ -99,6 +99,6 @@ optimized_logical_plan: | LogicalScan { table: mv, output_columns: [mv.v1], required_columns: [mv.v1, mv.v3], predicate: ((mv.v3 = 'world':Varchar) OR (mv.v3 = 'hello':Varchar)) } stream_plan: | - StreamMaterialize { columns: [v1, mv.v2(hidden), mv.v3(hidden)], pk_columns: [v1, mv.v2, mv.v3] } + StreamMaterialize { columns: [v1, mv.v2(hidden), mv.v3(hidden)], pk_columns: [v1, mv.v2, mv.v3], pk_conflict: "no check" } └─StreamFilter { predicate: ((mv.v3 = 'world':Varchar) OR (mv.v3 = 'hello':Varchar)) } └─StreamTableScan { table: mv, columns: [mv.v1, mv.v2, mv.v3], pk: [mv.v1, mv.v2, mv.v3], dist: UpstreamHashShard(mv.v1, mv.v2, mv.v3) } diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index 292d5dcbcb214..e68f9ac5ef81f 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -261,13 +261,13 @@ | └─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, v2] } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, v2], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } | └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output: [t1.v1, t1._row_id] } | ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } | └─StreamExchange { dist: Broadcast } - | └─StreamProject { exprs: [(now + '01:00:00':Interval) as $expr1], watermark_columns: [(now + '01:00:00':Interval)] } + | └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval)] } | └─StreamNow { output: [now] } └─StreamExchange { dist: HashShard(t2.v2) } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } @@ -301,12 +301,12 @@ LogicalFilter { predicate: (t1.v1 > (Now + '00:30:00':Interval)) } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: (t1.v2 > 5:Int32) } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output: [t1.v1, t1.v2, t1._row_id] } ├─StreamFilter { predicate: (t1.v2 > 5:Int32) } | └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(now + '00:30:00':Interval) as $expr1], watermark_columns: [(now + '00:30:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:30:00':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:30:00':Interval)] } └─StreamNow { output: [now] } - name: eq-predicate derived condition other side pushdown in inner join sql: | @@ -347,7 +347,7 @@ | └─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, v2] } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, v2], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } | └─StreamDynamicFilter { predicate: (t1.v1 > now), output: [t1.v1, t1._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index 879e6f59690d8..6c3d4f496d951 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -8,14 +8,14 @@ - sql: | select unnest(Array[1,2,3]); batch_plan: | - BatchProject { exprs: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } - └─BatchProjectSet { select_list: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } + BatchProject { exprs: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } + └─BatchProjectSet { select_list: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } └─BatchValues { rows: [[]] } - sql: | select unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchProject { exprs: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } - └─BatchProjectSet { select_list: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } + BatchProject { exprs: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } + └─BatchProjectSet { select_list: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } └─BatchValues { rows: [[]] } - sql: | create table t(x int[]); @@ -26,7 +26,7 @@ └─BatchProjectSet { select_list: [Unnest($0)] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } stream_plan: | - StreamMaterialize { columns: [projected_row_id(hidden), unnest, t._row_id(hidden)], pk_columns: [t._row_id, projected_row_id] } + StreamMaterialize { columns: [projected_row_id(hidden), unnest, t._row_id(hidden)], pk_columns: [t._row_id, projected_row_id], pk_conflict: "no check" } └─StreamProjectSet { select_list: [Unnest($0), $1] } └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: table functions used with usual expressions @@ -43,9 +43,9 @@ create table t(x int[]); select unnest(x), unnest(Array[1,2]) from t; batch_plan: | - BatchProject { exprs: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } + BatchProject { exprs: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } └─BatchExchange { order: [], dist: Single } - └─BatchProjectSet { select_list: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } + └─BatchProjectSet { select_list: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } - name: table functions as parameters of usual functions sql: | @@ -67,7 +67,7 @@ └─BatchProjectSet { select_list: [$0, $1, Unnest($0), Unnest($0)] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } stream_plan: | - StreamMaterialize { columns: [projected_row_id(hidden), a, b, t._row_id(hidden), projected_row_id#1(hidden)], pk_columns: [t._row_id, projected_row_id#1, projected_row_id] } + StreamMaterialize { columns: [projected_row_id(hidden), a, b, t._row_id(hidden), projected_row_id#1(hidden)], pk_columns: [t._row_id, projected_row_id#1, projected_row_id], pk_conflict: "no check" } └─StreamProjectSet { select_list: [($3 * $4), Unnest($1), $2, $0] } └─StreamProjectSet { select_list: [$0, $1, Unnest($0), Unnest($0)] } └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -94,7 +94,7 @@ └─BatchProjectSet { select_list: [Unnest($0)] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } stream_plan: | - StreamMaterialize { columns: [projected_row_id(hidden), unnest, t._row_id(hidden)], pk_columns: [t._row_id, projected_row_id], order_descs: [projected_row_id, t._row_id] } + StreamMaterialize { columns: [projected_row_id(hidden), unnest, t._row_id(hidden)], pk_columns: [t._row_id, projected_row_id], order_descs: [projected_row_id, t._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id] } └─StreamTopN { order: "[projected_row_id ASC]", limit: 1, offset: 0 } └─StreamExchange { dist: Single } @@ -114,7 +114,7 @@ └─BatchProjectSet { select_list: [Unnest($0)] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } stream_plan: | - StreamMaterialize { columns: [unnest], pk_columns: [unnest] } + StreamMaterialize { columns: [unnest], pk_columns: [unnest], pk_conflict: "no check" } └─StreamProject { exprs: [Unnest($0)] } └─StreamHashAgg { group_key: [Unnest($0)], aggs: [count] } └─StreamExchange { dist: HashShard(Unnest($0)) } @@ -130,7 +130,7 @@ └─BatchProjectSet { select_list: [Unnest($0)] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } stream_plan: | - StreamMaterialize { columns: [unnest, t._row_id(hidden), projected_row_id(hidden)], pk_columns: [t._row_id, projected_row_id] } + StreamMaterialize { columns: [unnest, t._row_id(hidden), projected_row_id(hidden)], pk_columns: [t._row_id, projected_row_id], pk_conflict: "no check" } └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } └─StreamFilter { predicate: (Unnest($0) > 1:Int32) } └─StreamProjectSet { select_list: [Unnest($0), $1] } @@ -153,7 +153,7 @@ └─BatchProjectSet { select_list: [Unnest($0)] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } stream_plan: | - StreamMaterialize { columns: [unnest, t._row_id(hidden), projected_row_id(hidden), t._row_id#1(hidden), projected_row_id#1(hidden), Unnest($0)(hidden)], pk_columns: [t._row_id, projected_row_id, t._row_id#1, projected_row_id#1, unnest, Unnest($0)] } + StreamMaterialize { columns: [unnest, t._row_id(hidden), projected_row_id(hidden), t._row_id#1(hidden), projected_row_id#1(hidden), Unnest($0)(hidden)], pk_columns: [t._row_id, projected_row_id, t._row_id#1, projected_row_id#1, unnest, Unnest($0)], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id, Unnest($0)] } ├─StreamExchange { dist: HashShard(Unnest($0)) } | └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } diff --git a/src/frontend/planner_test/tests/testdata/range_scan.yaml b/src/frontend/planner_test/tests/testdata/range_scan.yaml index df8dbca94c355..4c6678d46ccbd 100644 --- a/src/frontend/planner_test/tests/testdata/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/range_scan.yaml @@ -87,7 +87,7 @@ SELECT * FROM orders_count_by_user WHERE user_id > 42 AND date = 1111 AND 2>1 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (2:Int32 > 1:Int32) AND (orders_count_by_user.date = 1111:Int32) } + └─BatchFilter { predicate: true:Boolean AND (orders_count_by_user.date = 1111:Int32) } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id > Int64(42)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv @@ -95,7 +95,7 @@ SELECT * FROM orders_count_by_user WHERE date > 1111 AND user_id = 42 AND 5<6 AND date <= 6666 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (5:Int32 < 6:Int32) } + └─BatchFilter { predicate: true:Boolean } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id = Int64(42) AND orders_count_by_user.date > Int32(1111) AND orders_count_by_user.date <= Int32(6666)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index c55790c91f452..868ccc0c02c71 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -38,7 +38,7 @@ └─BatchFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─BatchSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [cnt], pk_columns: [] } + StreamMaterialize { columns: [cnt], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count)] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum0(count)] } └─StreamExchange { dist: Single } @@ -48,7 +48,7 @@ | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } | └─StreamProject { exprs: [id, initial_bid, _row_id] } - | └─StreamShare { id = 595 } + | └─StreamShare { id = 797 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 10 } @@ -57,7 +57,7 @@ └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 595 } + └─StreamShare { id = 797 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 10 } @@ -115,7 +115,7 @@ └─BatchProject { exprs: [auction, date_time] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [window_start, auction, window_start#1] } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [window_start, auction, window_start#1], pk_conflict: "no check" } └─StreamProject { exprs: [auction, count, window_start, window_start] } └─StreamFilter { predicate: (count >= max(count)) } └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } @@ -125,7 +125,7 @@ | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamShare { id = 854 } + | └─StreamShare { id = 1066 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -137,7 +137,7 @@ └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamShare { id = 854 } + └─StreamShare { id = 1066 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -146,11 +146,11 @@ create table t(a int, b int); with cte as (select count(*) from t) select * from cte union all select * from cte; stream_plan: | - StreamMaterialize { columns: [count, 0:Int32(hidden)], pk_columns: [0:Int32] } + StreamMaterialize { columns: [count, 0:Int32(hidden)], pk_columns: [0:Int32], pk_conflict: "no check" } └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(0:Int32) } | └─StreamProject { exprs: [sum0(count), 0:Int32] } - | └─StreamShare { id = 243 } + | └─StreamShare { id = 331 } | └─StreamProject { exprs: [sum0(count)] } | └─StreamGlobalSimpleAgg { aggs: [count, sum0(count)] } | └─StreamExchange { dist: Single } @@ -158,7 +158,7 @@ | └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(1:Int32) } └─StreamProject { exprs: [sum0(count), 1:Int32] } - └─StreamShare { id = 243 } + └─StreamShare { id = 331 } └─StreamProject { exprs: [sum0(count)] } └─StreamGlobalSimpleAgg { aggs: [count, sum0(count)] } └─StreamExchange { dist: Single } @@ -169,7 +169,7 @@ create table t(a int, b int); with cte as (select count(*) from t) select * from cte union all select * from cte; stream_plan: | - StreamMaterialize { columns: [count, 0:Int32(hidden)], pk_columns: [0:Int32] } + StreamMaterialize { columns: [count, 0:Int32(hidden)], pk_columns: [0:Int32], pk_conflict: "no check" } └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(0:Int32) } | └─StreamProject { exprs: [sum0(count), 0:Int32] } @@ -190,7 +190,7 @@ set rw_enable_share_plan=false; select count(*) cnt from auction A join auction B on A.id = B.id; stream_plan: | - StreamMaterialize { columns: [cnt], pk_columns: [] } + StreamMaterialize { columns: [cnt], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count)] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum0(count)] } └─StreamExchange { dist: Single } @@ -198,13 +198,13 @@ └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id, id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } - | └─StreamShare { id = 177 } + | └─StreamShare { id = 251 } | └─StreamProject { exprs: [id, _row_id] } | └─StreamRowIdGen { row_id_index: 10 } | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } └─StreamExchange { dist: HashShard(id) } └─StreamProject { exprs: [id, _row_id] } - └─StreamShare { id = 177 } + └─StreamShare { id = 251 } └─StreamProject { exprs: [id, _row_id] } └─StreamRowIdGen { row_id_index: 10 } └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/shared_views.yaml b/src/frontend/planner_test/tests/testdata/shared_views.yaml index 2b4d641ebfafa..ff9326cbee7a8 100644 --- a/src/frontend/planner_test/tests/testdata/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/shared_views.yaml @@ -22,11 +22,11 @@ └─LogicalFilter { predicate: (t1.y > 0:Int32) } └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden), $expr90(hidden)], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, $expr90, z, a] } + StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden), $expr90(hidden)], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, $expr90, z, a], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x, $expr1] } ├─StreamExchange { dist: HashShard($expr1) } | └─StreamProject { exprs: [$expr1, t1._row_id] } - | └─StreamShare { id = 250 } + | └─StreamShare { id = 348 } | └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } | └─StreamFilter { predicate: (t1.y > 0:Int32) } | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } @@ -37,7 +37,7 @@ | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [$expr1, t1._row_id] } - └─StreamShare { id = 250 } + └─StreamShare { id = 348 } └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } └─StreamFilter { predicate: (t1.y > 0:Int32) } └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml index c5ffc5cb34f78..5cfb004b9e9fb 100644 --- a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml @@ -16,13 +16,13 @@ └─BatchSimpleAgg { aggs: [max(s.v)] } └─BatchScan { table: s, columns: [s.v], distribution: Single } stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(s.v)] } └─StreamGlobalSimpleAgg { aggs: [count, max(s.v)] } └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(s.v)] } StreamGlobalSimpleAgg { aggs: [count, max(s.v)] } @@ -44,13 +44,13 @@ └─BatchSimpleAgg { aggs: [sum(s.v)] } └─BatchScan { table: s, columns: [s.v], distribution: Single } stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(s.v)] } └─StreamGlobalSimpleAgg { aggs: [count, sum(s.v)] } └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(s.v)] } StreamGlobalSimpleAgg { aggs: [count, sum(s.v)] } @@ -71,13 +71,13 @@ └─BatchSimpleAgg { aggs: [count(s.v)] } └─BatchScan { table: s, columns: [s.v], distribution: Single } stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [count(s.v)] } └─StreamGlobalSimpleAgg { aggs: [count, count(s.v)] } └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(s.v)] } StreamGlobalSimpleAgg { aggs: [count, count(s.v)] } @@ -99,14 +99,14 @@ └─BatchProject { exprs: [s.s, ',':Varchar, s.v] } └─BatchScan { table: s, columns: [s.v, s.s], distribution: Single } stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } └─StreamGlobalSimpleAgg { aggs: [count, string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } └─StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } └─StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } StreamGlobalSimpleAgg { aggs: [count, string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } @@ -130,7 +130,7 @@ └─BatchSimpleAgg { aggs: [max(t.v)] } └─BatchScan { table: t, columns: [t.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(t.v))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t.v))] } └─StreamExchange { dist: Single } @@ -139,7 +139,7 @@ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(max(t.v))] } StreamGlobalSimpleAgg { aggs: [count, max(max(t.v))] } @@ -147,17 +147,17 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.v)] } + StreamHashAgg { group_key: [$expr3], aggs: [count, max(t.v)] } result table: 3, state tables: [2] - StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } + StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } Upstream BatchPlanNode - Table 0 { columns: [max(t_v), $expr1], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [max(t_v), $expr3], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [count, max(max(t_v))], primary key: [], value indices: [0, 1], distribution key: [] } - Table 2 { columns: [$expr1, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr1, count, max(t_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr3, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr3, count, max(t_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_on_AO before: @@ -165,7 +165,7 @@ sql: | select max(v) as a1 from AO; stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(ao.v))] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(max(ao.v))] } └─StreamExchange { dist: Single } @@ -173,7 +173,7 @@ └─StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(max(ao.v))] } StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(max(ao.v))] } @@ -199,7 +199,7 @@ └─BatchSimpleAgg { aggs: [sum(t.v)] } └─BatchScan { table: t, columns: [t.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v))] } └─StreamExchange { dist: Single } @@ -207,7 +207,7 @@ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(sum(t.v))] } StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v))] } @@ -228,7 +228,7 @@ sql: | select sum(v) as a1 from AO; stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(ao.v))] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum(sum(ao.v))] } └─StreamExchange { dist: Single } @@ -236,7 +236,7 @@ └─StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(sum(ao.v))] } StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum(sum(ao.v))] } @@ -262,7 +262,7 @@ └─BatchSimpleAgg { aggs: [count(t.v)] } └─BatchScan { table: t, columns: [t.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count(t.v))] } └─StreamGlobalSimpleAgg { aggs: [count, sum0(count(t.v))] } └─StreamExchange { dist: Single } @@ -270,7 +270,7 @@ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum0(count(t.v))] } StreamGlobalSimpleAgg { aggs: [count, sum0(count(t.v))] } @@ -291,7 +291,7 @@ sql: | select count(v) as a1 from AO; stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count(ao.v))] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum0(count(ao.v))] } └─StreamExchange { dist: Single } @@ -299,7 +299,7 @@ └─StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum0(count(ao.v))] } StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum0(count(ao.v))] } @@ -325,7 +325,7 @@ └─BatchProject { exprs: [t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.o, t.s], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } └─StreamGlobalSimpleAgg { aggs: [count, string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } └─StreamExchange { dist: Single } @@ -333,7 +333,7 @@ └─StreamTableScan { table: t, columns: [t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } StreamGlobalSimpleAgg { aggs: [count, string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } @@ -355,7 +355,7 @@ sql: | select string_agg(s, ',' order by o) as a1 from AO; stream_plan: | - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } └─StreamExchange { dist: Single } @@ -363,7 +363,7 @@ └─StreamTableScan { table: ao, columns: [ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1], pk_columns: [] } + StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } StreamAppendOnlyGlobalSimpleAgg { aggs: [count, string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } @@ -389,7 +389,7 @@ └─BatchSimpleAgg { aggs: [max(t.v), count(t.v)] } └─BatchScan { table: t, columns: [t.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(t.v)), sum0(count(t.v))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t.v)), sum0(count(t.v))] } └─StreamExchange { dist: Single } @@ -398,7 +398,7 @@ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(max(t.v)), sum0(count(t.v))] } StreamGlobalSimpleAgg { aggs: [count, max(max(t.v)), sum0(count(t.v))] } @@ -406,17 +406,17 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.v), count(t.v)] } + StreamHashAgg { group_key: [$expr3], aggs: [count, max(t.v), count(t.v)] } result table: 3, state tables: [2] - StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } + StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } Upstream BatchPlanNode - Table 0 { columns: [max(t_v), $expr1], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [max(t_v), $expr3], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [count, max(max(t_v)), sum0(count(t_v))], primary key: [], value indices: [0, 1, 2], distribution key: [] } - Table 2 { columns: [$expr1, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr1, count, max(t_v), count(t_v)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr3, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr3, count, max(t_v), count(t_v)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_count_on_AO before: @@ -424,7 +424,7 @@ sql: | select max(v) as a1, count(v) as a2 from AO; stream_plan: | - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(ao.v)), sum0(count(ao.v))] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(max(ao.v)), sum0(count(ao.v))] } └─StreamExchange { dist: Single } @@ -432,7 +432,7 @@ └─StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(max(ao.v)), sum0(count(ao.v))] } StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(max(ao.v)), sum0(count(ao.v))] } @@ -458,7 +458,7 @@ └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } └─StreamGlobalSimpleAgg { aggs: [count, count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } └─StreamExchange { dist: Single } @@ -466,7 +466,7 @@ └─StreamTableScan { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } StreamGlobalSimpleAgg { aggs: [count, count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } @@ -488,7 +488,7 @@ sql: | select count(v) as a1, string_agg(s, ',' order by o) as a2 from AO; stream_plan: | - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } └─StreamExchange { dist: Single } @@ -496,7 +496,7 @@ └─StreamTableScan { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } StreamAppendOnlyGlobalSimpleAgg { aggs: [count, count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } @@ -522,7 +522,7 @@ └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } └─StreamGlobalSimpleAgg { aggs: [count, max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } └─StreamExchange { dist: Single } @@ -530,7 +530,7 @@ └─StreamTableScan { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } StreamGlobalSimpleAgg { aggs: [count, max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } @@ -553,7 +553,7 @@ sql: | select max(v) as a1, string_agg(s, ',' order by o) as a2 from AO; stream_plan: | - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } └─StreamExchange { dist: Single } @@ -561,7 +561,7 @@ └─StreamTableScan { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, a2], pk_columns: [] } + StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } @@ -588,14 +588,14 @@ └─BatchExchange { order: [], dist: HashShard(t.k) } └─BatchScan { table: t, columns: [t.k, t.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k] } + StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } └─StreamProject { exprs: [max(t.v), t.k] } └─StreamHashAgg { group_key: [t.k], aggs: [count, max(t.v)] } └─StreamExchange { dist: HashShard(t.k) } └─StreamTableScan { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k] } + StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(t.v), t.k] } StreamHashAgg { group_key: [t.k], aggs: [count, max(t.v)] } @@ -621,13 +621,13 @@ └─BatchSortAgg { group_key: [tk.k], aggs: [max(tk.v)] } └─BatchScan { table: tk, columns: [tk.k, tk.v], distribution: UpstreamHashShard(tk.k) } stream_plan: | - StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k] } + StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } └─StreamProject { exprs: [max(tk.v), tk.k] } └─StreamHashAgg { group_key: [tk.k], aggs: [count, max(tk.v)] } └─StreamTableScan { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k] } + StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(tk.v), tk.k] } StreamHashAgg { group_key: [tk.k], aggs: [count, max(tk.v)] } @@ -651,14 +651,14 @@ └─BatchExchange { order: [], dist: HashShard(s.k) } └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } stream_plan: | - StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k] } + StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } └─StreamProject { exprs: [max(s.v), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [count, max(s.v)] } └─StreamExchange { dist: HashShard(s.k) } └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k] } + StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(s.v), s.k] } StreamHashAgg { group_key: [s.k], aggs: [count, max(s.v)] } @@ -679,14 +679,14 @@ sql: | select max(v) as a1 from AO group by k; stream_plan: | - StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k] } + StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } └─StreamProject { exprs: [max(ao.v), ao.k] } └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, max(ao.v)] } └─StreamExchange { dist: HashShard(ao.k) } └─StreamTableScan { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k] } + StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(ao.v), ao.k] } StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, max(ao.v)] } @@ -712,14 +712,14 @@ └─BatchExchange { order: [], dist: HashShard(t.k) } └─BatchScan { table: t, columns: [t.k, t.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k] } + StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } └─StreamProject { exprs: [sum(t.v), t.k] } └─StreamHashAgg { group_key: [t.k], aggs: [count, sum(t.v)] } └─StreamExchange { dist: HashShard(t.k) } └─StreamTableScan { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k] } + StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(t.v), t.k] } StreamHashAgg { group_key: [t.k], aggs: [count, sum(t.v)] } @@ -744,13 +744,13 @@ └─BatchSortAgg { group_key: [tk.k], aggs: [sum(tk.v)] } └─BatchScan { table: tk, columns: [tk.k, tk.v], distribution: UpstreamHashShard(tk.k) } stream_plan: | - StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k] } + StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } └─StreamProject { exprs: [sum(tk.v), tk.k] } └─StreamHashAgg { group_key: [tk.k], aggs: [count, sum(tk.v)] } └─StreamTableScan { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k] } + StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(tk.v), tk.k] } StreamHashAgg { group_key: [tk.k], aggs: [count, sum(tk.v)] } @@ -773,14 +773,14 @@ └─BatchExchange { order: [], dist: HashShard(s.k) } └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } stream_plan: | - StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k] } + StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } └─StreamProject { exprs: [sum(s.v), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [count, sum(s.v)] } └─StreamExchange { dist: HashShard(s.k) } └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k] } + StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(s.v), s.k] } StreamHashAgg { group_key: [s.k], aggs: [count, sum(s.v)] } @@ -800,14 +800,14 @@ sql: | select sum(v) as a1 from AO group by k; stream_plan: | - StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k] } + StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } └─StreamProject { exprs: [sum(ao.v), ao.k] } └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, sum(ao.v)] } └─StreamExchange { dist: HashShard(ao.k) } └─StreamTableScan { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k] } + StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(ao.v), ao.k] } StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, sum(ao.v)] } @@ -833,14 +833,14 @@ └─BatchExchange { order: [], dist: HashShard(t.k) } └─BatchScan { table: t, columns: [t.k, t.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k] } + StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } └─StreamProject { exprs: [count(t.v), t.k] } └─StreamHashAgg { group_key: [t.k], aggs: [count, count(t.v)] } └─StreamExchange { dist: HashShard(t.k) } └─StreamTableScan { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k] } + StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(t.v), t.k] } StreamHashAgg { group_key: [t.k], aggs: [count, count(t.v)] } @@ -865,13 +865,13 @@ └─BatchSortAgg { group_key: [tk.k], aggs: [count(tk.v)] } └─BatchScan { table: tk, columns: [tk.k, tk.v], distribution: UpstreamHashShard(tk.k) } stream_plan: | - StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k] } + StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } └─StreamProject { exprs: [count(tk.v), tk.k] } └─StreamHashAgg { group_key: [tk.k], aggs: [count, count(tk.v)] } └─StreamTableScan { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k] } + StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(tk.v), tk.k] } StreamHashAgg { group_key: [tk.k], aggs: [count, count(tk.v)] } @@ -894,14 +894,14 @@ └─BatchExchange { order: [], dist: HashShard(s.k) } └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } stream_plan: | - StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k] } + StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } └─StreamProject { exprs: [count(s.v), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [count, count(s.v)] } └─StreamExchange { dist: HashShard(s.k) } └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k] } + StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(s.v), s.k] } StreamHashAgg { group_key: [s.k], aggs: [count, count(s.v)] } @@ -921,14 +921,14 @@ sql: | select count(v) as a1 from AO group by k; stream_plan: | - StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k] } + StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } └─StreamProject { exprs: [count(ao.v), ao.k] } └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, count(ao.v)] } └─StreamExchange { dist: HashShard(ao.k) } └─StreamTableScan { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k] } + StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(ao.v), ao.k] } StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, count(ao.v)] } @@ -955,7 +955,7 @@ └─BatchProject { exprs: [t.k, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.k, t.o, t.s], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k] } + StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), t.k] } └─StreamHashAgg { group_key: [t.k], aggs: [count, string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } └─StreamExchange { dist: HashShard(t.k) } @@ -963,7 +963,7 @@ └─StreamTableScan { table: t, columns: [t.k, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k] } + StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), t.k] } StreamHashAgg { group_key: [t.k], aggs: [count, string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } @@ -991,14 +991,14 @@ └─BatchProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o] } └─BatchScan { table: tk, columns: [tk.k, tk.o, tk.s], distribution: UpstreamHashShard(tk.k) } stream_plan: | - StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k] } + StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), tk.k] } └─StreamHashAgg { group_key: [tk.k], aggs: [count, string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST))] } └─StreamProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o, tk.t._row_id] } └─StreamTableScan { table: tk, columns: [tk.k, tk.o, tk.s, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k] } + StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), tk.k] } StreamHashAgg { group_key: [tk.k], aggs: [count, string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST))] } @@ -1024,7 +1024,7 @@ └─BatchProject { exprs: [s.k, s.s, ',':Varchar, s.o] } └─BatchScan { table: s, columns: [s.k, s.o, s.s], distribution: Single } stream_plan: | - StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k] } + StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [count, string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST))] } └─StreamExchange { dist: HashShard(s.k) } @@ -1032,7 +1032,7 @@ └─StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k] } + StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), s.k] } StreamHashAgg { group_key: [s.k], aggs: [count, string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST))] } @@ -1054,7 +1054,7 @@ sql: | select string_agg(s, ',' order by o) as a1 from AO group by k; stream_plan: | - StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k] } + StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), ao.k] } └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } └─StreamExchange { dist: HashShard(ao.k) } @@ -1062,7 +1062,7 @@ └─StreamTableScan { table: ao, columns: [ao.k, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k] } + StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), ao.k] } StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } diff --git a/src/frontend/planner_test/tests/testdata/struct_query.yaml b/src/frontend/planner_test/tests/testdata/struct_query.yaml index 8d90395ecceb9..3ba6dbe2fcb8c 100644 --- a/src/frontend/planner_test/tests/testdata/struct_query.yaml +++ b/src/frontend/planner_test/tests/testdata/struct_query.yaml @@ -6,7 +6,7 @@ BatchExchange { order: [], dist: Single } └─BatchScan { table: t, columns: [t.country], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [country, t._row_id(hidden)], pk_columns: [t._row_id] } + StreamMaterialize { columns: [country, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } └─StreamTableScan { table: t, columns: [t.country, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } create_source: row_format: protobuf diff --git a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml index c6b3a39fe411d..385323133d317 100644 --- a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml +++ b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml @@ -29,6 +29,6 @@ Feature is not yet implemented: Only boolean literals are supported in `current_schemas`. 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 current_timestamp; + select current_timestamp > now() - interval '10 minutes'; batch_plan: | - BatchValues { rows: [[Now('2021-04-01 00:00:00+00:00':Timestamptz)]] } + BatchValues { rows: [[true:Boolean]] } diff --git a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml index e840753ab2250..13850fe1188cb 100644 --- a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml @@ -4,10 +4,10 @@ create table t1 (ts timestamp with time zone); select * from t1 where ts + interval '1 hour' > now(); stream_plan: | - StreamMaterialize { columns: [ts, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [ts, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t1.ts, t1._row_id] } └─StreamDynamicFilter { predicate: ($expr1 > now), output: [t1.ts, $expr1, t1._row_id] } - ├─StreamProject { exprs: [t1.ts, (t1.ts + '01:00:00':Interval) as $expr1, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval) as $expr1, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamNow { output: [now] } @@ -16,10 +16,10 @@ create table t1 (ts timestamp with time zone, time_to_live interval); select * from t1 where ts + time_to_live * 1.5 > now(); stream_plan: | - StreamMaterialize { columns: [ts, time_to_live, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [ts, time_to_live, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t1.ts, t1.time_to_live, t1._row_id] } └─StreamDynamicFilter { predicate: ($expr1 > now), output: [t1.ts, t1.time_to_live, $expr1, t1._row_id] } - ├─StreamProject { exprs: [t1.ts, t1.time_to_live, (t1.ts + (t1.time_to_live * 1.5:Decimal)) as $expr1, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, t1.time_to_live, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + DateTrunc('day':Varchar, (t1.time_to_live * 1.5:Decimal))), 'UTC':Varchar) + ((t1.time_to_live * 1.5:Decimal) - DateTrunc('day':Varchar, (t1.time_to_live * 1.5:Decimal)))) as $expr1, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.time_to_live, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamNow { output: [now] } @@ -28,13 +28,13 @@ create table t1 (ts timestamp with time zone, additional_time_to_live interval); select * from t1 where now() - interval '15 minutes' < ts + additional_time_to_live * 1.5; stream_plan: | - StreamMaterialize { columns: [ts, additional_time_to_live, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [ts, additional_time_to_live, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, t1._row_id] } └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output: [t1.ts, t1.additional_time_to_live, $expr1, t1._row_id] } - ├─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, (t1.ts + (t1.additional_time_to_live * 1.5:Decimal)) as $expr1, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + DateTrunc('day':Varchar, (t1.additional_time_to_live * 1.5:Decimal))), 'UTC':Varchar) + ((t1.additional_time_to_live * 1.5:Decimal) - DateTrunc('day':Varchar, (t1.additional_time_to_live * 1.5:Decimal)))) as $expr1, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.additional_time_to_live, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(now - '00:15:00':Interval) as $expr2], watermark_columns: [(now - '00:15:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:15:00':Interval) as $expr2], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:15:00':Interval)] } └─StreamNow { output: [now] } - name: Temporal filter fails without `now()` in lower bound sql: |- @@ -46,13 +46,13 @@ create table t1 (ts timestamp with time zone); select * from t1 where ts < now() - interval '1 hour' and ts >= now() - interval '2 hour'; stream_plan: | - StreamMaterialize { columns: [ts, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [ts, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (t1.ts < $expr2), output: [t1.ts, t1._row_id] } ├─StreamDynamicFilter { predicate: (t1.ts >= $expr1), output: [t1.ts, t1._row_id] } | ├─StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } | └─StreamExchange { dist: Broadcast } - | └─StreamProject { exprs: [(now - '02:00:00':Interval) as $expr1], watermark_columns: [(now - '02:00:00':Interval)] } + | └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '02:00:00':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '02:00:00':Interval)] } | └─StreamNow { output: [now] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(now - '01:00:00':Interval) as $expr2], watermark_columns: [(now - '01:00:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '01:00:00':Interval) as $expr2], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '01:00:00':Interval)] } └─StreamNow { output: [now] } diff --git a/src/frontend/planner_test/tests/testdata/time_window.yaml b/src/frontend/planner_test/tests/testdata/time_window.yaml index df3dd8c1946da..1575c405567f3 100644 --- a/src/frontend/planner_test/tests/testdata/time_window.yaml +++ b/src/frontend/planner_test/tests/testdata/time_window.yaml @@ -64,7 +64,7 @@ └─LogicalHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: all } └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [id, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start, window_end] } + StreamMaterialize { columns: [id, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start, window_end], pk_conflict: "no check" } └─StreamHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.created_at, window_start, window_end, t1._row_id] } └─StreamTableScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | @@ -75,7 +75,7 @@ └─LogicalHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: all } └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [id, created_at, window_start, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start] } + StreamMaterialize { columns: [id, created_at, window_start, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start], pk_conflict: "no check" } └─StreamHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.created_at, window_start, t1._row_id] } └─StreamTableScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | @@ -86,7 +86,7 @@ └─LogicalHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: all } └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [id, created_at, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_end] } + StreamMaterialize { columns: [id, created_at, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_end], pk_conflict: "no check" } └─StreamHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.created_at, window_end, t1._row_id] } └─StreamTableScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | @@ -101,7 +101,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t1, columns: [t1.id, t1.created_at], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [id, created_at, window_start(hidden), t1._row_id(hidden)], pk_columns: [t1._row_id, window_start] } + StreamMaterialize { columns: [id, created_at, window_start(hidden), t1._row_id(hidden)], pk_columns: [t1._row_id, window_start], pk_conflict: "no check" } └─StreamHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.created_at, window_start, t1._row_id] } └─StreamTableScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | @@ -116,7 +116,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t1, columns: [t1.id, t1.created_at], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [id, created_at, window_start(hidden), t1._row_id(hidden)], pk_columns: [t1._row_id, window_start] } + StreamMaterialize { columns: [id, created_at, window_start(hidden), t1._row_id(hidden)], pk_columns: [t1._row_id, window_start], pk_conflict: "no check" } └─StreamHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.created_at, window_start, t1._row_id] } └─StreamTableScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | @@ -136,7 +136,7 @@ └─BatchExchange { order: [], dist: HashShard(t.v1) } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, window_end, avg], pk_columns: [v1, window_end] } + StreamMaterialize { columns: [v1, window_end, avg], pk_columns: [v1, window_end], pk_conflict: "no check" } └─StreamProject { exprs: [t.v1, window_end, (sum(t.v3) / count(t.v3)::Float64) as $expr1] } └─StreamHashAgg { group_key: [t.v1, window_end], aggs: [count, sum(t.v3), count(t.v3)] } └─StreamExchange { dist: HashShard(t.v1, window_end) } @@ -159,7 +159,7 @@ └─BatchFilter { predicate: (t1.v1 >= 10:Int32) } └─BatchScan { table: t1, columns: [t1.id, t1.v1, t1.created_at], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [id, v1, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id] } + StreamMaterialize { columns: [id, v1, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t1.id, t1.v1, t1.created_at, TumbleStart(t1.created_at, '3 days':Interval) as $expr1, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr2, t1._row_id] } └─StreamFilter { predicate: (t1.v1 >= 10:Int32) } └─StreamTableScan { table: t1, columns: [t1.id, t1.v1, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } @@ -180,7 +180,7 @@ └─BatchFilter { predicate: (t1.v1 >= 10:Int32) } └─BatchScan { table: t1, columns: [t1.id, t1.v1, t1.created_at], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [id, v1, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start, window_end] } + StreamMaterialize { columns: [id, v1, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start, window_end], pk_conflict: "no check" } └─StreamHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.v1, t1.created_at, window_start, window_end, t1._row_id] } └─StreamFilter { predicate: (t1.v1 >= 10:Int32) } └─StreamTableScan { table: t1, columns: [t1.id, t1.v1, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 8b0e585c11a21..da804e91ac887 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -136,33 +136,33 @@ └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr2, lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus] } + StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus], pk_conflict: "no check" } └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr3, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr4, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr5, count] } └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus] } + StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr335), sum($expr336), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr340, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr341, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr342, count] } - StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr335), sum($expr336), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr358), sum($expr359), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr360, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr361, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr362, count] } + StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr358), sum($expr359), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr335, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr336, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr358, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr359, lineitem.l_discount, 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) } Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, count, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr335), sum($expr336), sum(lineitem_l_quantity)_0, count(lineitem_l_quantity), sum(lineitem_l_extendedprice)_0, count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0, 1] } + Table 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, count, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr358), sum($expr359), sum(lineitem_l_quantity)_0, count(lineitem_l_quantity), sum(lineitem_l_extendedprice)_0, count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0, 1] } Table 4294967294 { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: tpch_q2 before: @@ -296,7 +296,7 @@ └─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) } stream_plan: | - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp.ps_partkey(hidden), partsupp.ps_suppkey(hidden), supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden), part.p_partkey(hidden), partsupp.ps_supplycost(hidden), min(partsupp.ps_supplycost)(hidden), nation.n_regionkey(hidden), region.r_regionkey(hidden)], pk_columns: [p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey], order_descs: [s_acctbal, n_name, s_name, p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey] } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp.ps_partkey(hidden), partsupp.ps_suppkey(hidden), supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden), part.p_partkey(hidden), partsupp.ps_supplycost(hidden), min(partsupp.ps_supplycost)(hidden), nation.n_regionkey(hidden), region.r_regionkey(hidden)], pk_columns: [p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey], order_descs: [s_acctbal, n_name, s_name, p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey], pk_conflict: "no check" } └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } └─StreamTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0 } └─StreamExchange { dist: Single } @@ -353,7 +353,7 @@ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp.ps_partkey(hidden), partsupp.ps_suppkey(hidden), supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden), part.p_partkey(hidden), partsupp.ps_supplycost(hidden), min(partsupp.ps_supplycost)(hidden), nation.n_regionkey(hidden), region.r_regionkey(hidden)], pk_columns: [p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey], order_descs: [s_acctbal, n_name, s_name, p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey] } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp.ps_partkey(hidden), partsupp.ps_suppkey(hidden), supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden), part.p_partkey(hidden), partsupp.ps_supplycost(hidden), min(partsupp.ps_supplycost)(hidden), nation.n_regionkey(hidden), region.r_regionkey(hidden)], pk_columns: [p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey], order_descs: [s_acctbal, n_name, s_name, p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } StreamTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0 } @@ -363,7 +363,7 @@ Fragment 1 StreamGroupTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0, group_key: [18] } state table: 1 - StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr1] } + StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr3] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, 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, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } left table: 2, right table 4, left degree table: 3, right degree table: 5, StreamExchange Hash([8]) from 2 @@ -479,8 +479,8 @@ Upstream BatchPlanNode - Table 0 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr1], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [] } - Table 1 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr1], primary key: [$18 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [16], vnode column idx: 18 } + Table 0 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr3], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [] } + Table 1 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr3], primary key: [$18 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [16], vnode column idx: 18 } Table 2 { columns: [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, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_supplycost, part_p_partkey_0, min(partsupp_ps_supplycost)], primary key: [$8 ASC, $0 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $15 ASC, $14 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [8] } Table 3 { columns: [nation_n_regionkey, part_p_partkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 4 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -592,7 +592,7 @@ └─BatchFilter { predicate: (lineitem.l_shipdate > '1995-03-29':Date) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], pk_columns: [l_orderkey, o_orderdate, o_shippriority], order_descs: [revenue, o_orderdate, l_orderkey, o_shippriority] } + StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], pk_columns: [l_orderkey, o_orderdate, o_shippriority], order_descs: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: "no check" } └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } └─StreamTopN { order: "[sum($expr1) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } └─StreamExchange { dist: Single } @@ -618,24 +618,24 @@ └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], pk_columns: [l_orderkey, o_orderdate, o_shippriority], order_descs: [revenue, o_orderdate, l_orderkey, o_shippriority] } + StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], pk_columns: [l_orderkey, o_orderdate, o_shippriority], order_descs: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_orderkey, sum($expr67), orders.o_orderdate, orders.o_shippriority] } - StreamTopN { order: "[sum($expr67) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority] } + StreamTopN { order: "[sum($expr73) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr67) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } + StreamGroupTopN { order: "[sum($expr73) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } state table: 1 - StreamProject { exprs: [lineitem.l_orderkey, sum($expr67), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr68] } - StreamProject { exprs: [lineitem.l_orderkey, sum($expr67), orders.o_orderdate, orders.o_shippriority] } - StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr67)] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr74] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority] } + StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr73)] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 Fragment 2 - StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr67, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr73, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([0]) from 3 @@ -667,9 +667,9 @@ Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_orderkey, sum($expr67), orders_o_orderdate, orders_o_shippriority, $expr68], primary key: [$1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [] } - Table 1 { columns: [lineitem_l_orderkey, sum($expr67), orders_o_orderdate, orders_o_shippriority, $expr68], primary key: [$4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 2, 3], vnode column idx: 4 } - Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, count, sum($expr67)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [lineitem_l_orderkey, sum($expr73), orders_o_orderdate, orders_o_shippriority, $expr74], primary key: [$1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [] } + Table 1 { columns: [lineitem_l_orderkey, sum($expr73), orders_o_orderdate, orders_o_shippriority, $expr74], primary key: [$4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 2, 3], vnode column idx: 4 } + Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, count, sum($expr73)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate, orders_o_shippriority, customer_c_custkey, orders_o_custkey], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -727,21 +727,21 @@ └─BatchHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey] } └─BatchFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [o_orderpriority, order_count], pk_columns: [o_orderpriority] } + StreamMaterialize { columns: [o_orderpriority, order_count], pk_columns: [o_orderpriority], pk_conflict: "no check" } └─StreamProject { exprs: [orders.o_orderpriority, count] } └─StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count, count] } └─StreamExchange { dist: HashShard(orders.o_orderpriority) } └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } | └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } @@ -749,7 +749,7 @@ └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [o_orderpriority, order_count], pk_columns: [o_orderpriority] } + StreamMaterialize { columns: [o_orderpriority, order_count], pk_columns: [o_orderpriority], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [orders.o_orderpriority, count] } StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count, count] } @@ -764,7 +764,7 @@ Fragment 2 StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + 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) } Upstream BatchPlanNode @@ -859,14 +859,14 @@ | | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], distribution: UpstreamHashShard(customer.c_custkey) } | | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } | | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } | └─BatchExchange { order: [], dist: HashShard(supplier.s_nationkey) } | └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey, lineitem.l_suppkey) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [n_name, revenue], pk_columns: [n_name], order_descs: [revenue, n_name] } + StreamMaterialize { columns: [n_name, revenue], pk_columns: [n_name], order_descs: [revenue, n_name], pk_conflict: "no check" } └─StreamProject { exprs: [nation.n_name, sum($expr1)] } └─StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr1)] } └─StreamExchange { dist: HashShard(nation.n_name) } @@ -884,7 +884,7 @@ | | | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } | | | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | | | └─StreamExchange { dist: HashShard(supplier.s_nationkey) } | | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } @@ -898,15 +898,15 @@ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [n_name, revenue], pk_columns: [n_name], order_descs: [revenue, n_name] } + StreamMaterialize { columns: [n_name, revenue], pk_columns: [n_name], order_descs: [revenue, n_name], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, sum($expr67)] } - StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr67)] } + StreamProject { exprs: [nation.n_name, sum($expr71)] } + StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr71)] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr67, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr71, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([3]) from 2 @@ -943,7 +943,7 @@ Fragment 7 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + 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) } Upstream BatchPlanNode @@ -970,7 +970,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, count, sum($expr67)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [nation_n_name, count, sum($expr71)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey], primary key: [$3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [3] } Table 2 { columns: [nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC], value indices: [11], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1020,35 +1020,35 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum($expr1)] } └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } + └─BatchFilter { 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:Int32) } └─BatchScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [revenue], pk_columns: [] } + StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum($expr1))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1))] } └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, 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 < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } + └─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:Int32) } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [revenue], pk_columns: [] } + StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [sum(sum($expr69))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr69))] } + StreamProject { exprs: [sum(sum($expr73))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr73))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr69)] } - StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr69, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr73)] } + StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr73, 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:Int32) } 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) } Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum($expr69))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr73))], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [revenue], primary key: [], value indices: [0], distribution key: [] } - id: tpch_q7 before: @@ -1145,7 +1145,7 @@ └─BatchFilter { predicate: (lineitem.l_shipdate >= '1983-01-01':Date) AND (lineitem.l_shipdate <= '2000-12-31':Date) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], pk_columns: [supp_nation, cust_nation, l_year] } + StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: "no check" } └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [count, sum($expr2)] } └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr1) } @@ -1175,15 +1175,15 @@ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], pk_columns: [supp_nation, cust_nation, l_year] } + StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, nation.n_name, $expr141, sum($expr142)] } - StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr141], aggs: [count, sum($expr142)] } + StreamProject { exprs: [nation.n_name, nation.n_name, $expr149, sum($expr150)] } + StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr149], aggs: [count, sum($expr150)] } result table: 0, state tables: [] StreamExchange Hash([0, 1, 2]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr141, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr142, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } + StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr149, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr150, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, @@ -1245,7 +1245,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, nation_n_name_0, $expr141, count, sum($expr142)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [nation_n_name, nation_n_name_0, $expr149, count, sum($expr150)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, customer_c_custkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $12 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [4] } Table 2 { columns: [customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [nation_n_nationkey, nation_n_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1354,7 +1354,7 @@ └─BatchSort { order: [$expr1 ASC] } └─BatchHashAgg { group_key: [$expr1], aggs: [sum($expr2), sum($expr3)] } └─BatchExchange { order: [], dist: HashShard($expr1) } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3] } └─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] } └─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] } @@ -1375,11 +1375,11 @@ └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year] } + StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year], pk_conflict: "no check" } └─StreamProject { exprs: [$expr1, RoundDigit((sum($expr2) / sum($expr3)), 6:Int32) as $expr4] } └─StreamHashAgg { group_key: [$expr1], aggs: [count, sum($expr2), sum($expr3)] } └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } | └─StreamHashJoin { 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, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } @@ -1416,15 +1416,15 @@ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year] } + StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [$expr274, RoundDigit((sum($expr275) / sum($expr276)), 6:Int32) as $expr278] } - StreamHashAgg { group_key: [$expr274], aggs: [count, sum($expr275), sum($expr276)] } + StreamProject { exprs: [$expr291, RoundDigit((sum($expr292) / sum($expr293)), 6:Int32) as $expr294] } + StreamHashAgg { group_key: [$expr291], aggs: [count, sum($expr292), sum($expr293)] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr274, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr275, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr276, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr291, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr292, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr293, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([4]) from 2 @@ -1511,7 +1511,7 @@ Upstream BatchPlanNode - Table 0 { columns: [$expr274, count, sum($expr275), sum($expr276)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [$expr291, count, sum($expr292), sum($expr293)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, nation_n_nationkey_0], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [4] } Table 2 { columns: [nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey_0, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [14], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1632,7 +1632,7 @@ └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [nation, o_year, sum_profit], pk_columns: [nation, o_year] } + StreamMaterialize { columns: [nation, o_year, sum_profit], pk_columns: [nation, o_year], pk_conflict: "no check" } └─StreamProject { exprs: [nation.n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } └─StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [count, sum($expr2)] } └─StreamExchange { dist: HashShard(nation.n_name, $expr1) } @@ -1662,15 +1662,15 @@ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [nation, o_year, sum_profit], pk_columns: [nation, o_year] } + StreamMaterialize { columns: [nation, o_year, sum_profit], pk_columns: [nation, o_year], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, $expr205, RoundDigit(sum($expr206), 2:Int32) as $expr208] } - StreamHashAgg { group_key: [nation.n_name, $expr205], aggs: [count, sum($expr206)] } + StreamProject { exprs: [nation.n_name, $expr218, RoundDigit(sum($expr219), 2:Int32) as $expr220] } + StreamHashAgg { group_key: [nation.n_name, $expr218], aggs: [count, sum($expr219)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr205, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr206, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } + StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr218, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr219, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } StreamHashJoin { 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, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -1732,7 +1732,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, $expr205, count, sum($expr206)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 0 { columns: [nation_n_name, $expr218, count, sum($expr219)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } Table 1 { columns: [lineitem_l_orderkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, partsupp_ps_supplycost, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey], primary key: [$0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0] } Table 2 { columns: [lineitem_l_orderkey, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1832,14 +1832,14 @@ | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], distribution: UpstreamHashShard(customer.c_custkey) } | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount] } └─BatchFilter { predicate: (lineitem.l_returnflag = 'R':Varchar) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_returnflag], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], pk_columns: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], order_descs: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment] } + StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], pk_columns: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], order_descs: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: "no check" } └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } └─StreamTopN { order: "[sum($expr1) DESC]", limit: 20, offset: 0 } └─StreamExchange { dist: Single } @@ -1858,7 +1858,7 @@ | | | └─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) } | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | └─StreamExchange { dist: HashShard(nation.n_nationkey) } | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } @@ -1868,24 +1868,24 @@ └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], pk_columns: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], order_descs: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment] } + StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], pk_columns: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], order_descs: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr67), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - StreamTopN { order: "[sum($expr67) DESC]", limit: 20, offset: 0 } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + StreamTopN { order: "[sum($expr73) DESC]", limit: 20, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr67) DESC]", limit: 20, offset: 0, group_key: [8] } + StreamGroupTopN { order: "[sum($expr73) DESC]", limit: 20, offset: 0, group_key: [8] } state table: 1 - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr67), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr68] } - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr67), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [count, sum($expr67)] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr74] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [count, sum($expr73)] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 2, 3, 4, 5, 6]) from 2 Fragment 2 - StreamProject { 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 $expr67, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { 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 $expr73, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } StreamHashJoin { 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, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([6]) from 3 @@ -1910,7 +1910,7 @@ Fragment 6 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + 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) } Upstream BatchPlanNode @@ -1927,9 +1927,9 @@ Upstream BatchPlanNode - Table 0 { columns: [customer_c_custkey, customer_c_name, sum($expr67), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr68], primary key: [$2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [] } - Table 1 { columns: [customer_c_custkey, customer_c_name, sum($expr67), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr68], primary key: [$8 ASC, $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0, 1, 3, 6, 4, 5, 7], vnode column idx: 8 } - Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, count, sum($expr67)], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } + Table 0 { columns: [customer_c_custkey, customer_c_name, sum($expr73), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr74], primary key: [$2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [] } + Table 1 { columns: [customer_c_custkey, customer_c_name, sum($expr73), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr74], primary key: [$8 ASC, $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0, 1, 3, 6, 4, 5, 7], vnode column idx: 8 } + Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, count, sum($expr73)], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } Table 3 { columns: [customer_c_custkey, customer_c_name, customer_c_address, customer_c_phone, customer_c_acctbal, customer_c_comment, orders_o_orderkey, nation_n_name, orders_o_custkey, customer_c_nationkey, nation_n_nationkey], primary key: [$6 ASC, $0 ASC, $8 ASC, $10 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [6] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -2036,7 +2036,7 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } └─BatchScan { table: partsupp, columns: [partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [ps_partkey, value], pk_columns: [ps_partkey], order_descs: [value, ps_partkey] } + StreamMaterialize { columns: [ps_partkey, value], pk_columns: [ps_partkey], order_descs: [value, ps_partkey], pk_conflict: "no check" } └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [partsupp.ps_partkey, sum($expr1), sum($expr1)] } ├─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1), sum($expr1)] } @@ -2073,19 +2073,19 @@ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [ps_partkey, value], pk_columns: [ps_partkey], order_descs: [value, ps_partkey] } + StreamMaterialize { columns: [ps_partkey, value], pk_columns: [ps_partkey], order_descs: [value, ps_partkey], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [partsupp.ps_partkey, sum($expr199)] } - StreamDynamicFilter { predicate: (sum($expr199) > $expr202), output: [partsupp.ps_partkey, sum($expr199), sum($expr199)] } + StreamProject { exprs: [partsupp.ps_partkey, sum($expr212)] } + StreamDynamicFilter { predicate: (sum($expr212) > $expr214), output: [partsupp.ps_partkey, sum($expr212), sum($expr212)] } left table: 0, right table 1 - StreamProject { exprs: [partsupp.ps_partkey, sum($expr199), sum($expr199)] } - StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr199), sum($expr199)] } + StreamProject { exprs: [partsupp.ps_partkey, sum($expr212), sum($expr212)] } + StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr212), sum($expr212)] } result table: 2, state tables: [] StreamExchange Hash([0]) from 1 StreamExchange Broadcast from 6 Fragment 1 - StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr199, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr212, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([3]) from 2 @@ -2115,14 +2115,14 @@ BatchPlanNode Fragment 6 - StreamProject { exprs: [(sum(sum($expr200)) * 0.0001000000:Decimal) as $expr202] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr200))] } + StreamProject { exprs: [(sum(sum($expr213)) * 0.0001000000:Decimal) as $expr214] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr213))] } result table: 11, state tables: [] StreamExchange Single from 7 Fragment 7 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr200)] } - StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr200, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr213)] } + StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr213, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } left table: 12, right table 14, left degree table: 13, right degree table: 15, StreamExchange Hash([2]) from 8 @@ -2151,9 +2151,9 @@ Upstream BatchPlanNode - Table 0 { columns: [partsupp_ps_partkey, sum($expr199), sum($expr199)_0], primary key: [$2 ASC, $0 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 1 { columns: [$expr202], primary key: [], value indices: [0], distribution key: [] } - Table 2 { columns: [partsupp_ps_partkey, count, sum($expr199), sum($expr199)_0], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [partsupp_ps_partkey, sum($expr212), sum($expr212)_0], primary key: [$2 ASC, $0 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 1 { columns: [$expr214], primary key: [], value indices: [0], distribution key: [] } + Table 2 { columns: [partsupp_ps_partkey, count, sum($expr212), sum($expr212)_0], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 3 { columns: [partsupp_ps_partkey, partsupp_ps_availqty, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey], primary key: [$3 ASC, $0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [3] } Table 4 { columns: [supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 5 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -2162,7 +2162,7 @@ Table 8 { columns: [partsupp_ps_suppkey, partsupp_ps_partkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 9 { columns: [supplier_s_suppkey, supplier_s_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 10 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 11 { columns: [count, sum(sum($expr200))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [count, sum(sum($expr213))], primary key: [], value indices: [0, 1], distribution key: [] } Table 12 { columns: [partsupp_ps_availqty, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey], primary key: [$2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2] } Table 13 { columns: [supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 14 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -2229,10 +2229,10 @@ | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode] } - └─BatchFilter { 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 < ('1994-01-01':Date + '1 year':Interval)) } + └─BatchFilter { 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) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode] } + StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode], pk_conflict: "no check" } └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr1), sum($expr2)] } └─StreamExchange { dist: HashShard(lineitem.l_shipmode) } @@ -2242,19 +2242,19 @@ | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─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 < ('1994-01-01':Date + '1 year':Interval)) } + └─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) } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode] } + StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_shipmode, sum($expr133), sum($expr134)] } - StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr133), sum($expr134)] } + StreamProject { exprs: [lineitem.l_shipmode, sum($expr141), sum($expr142)] } + StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr141), sum($expr142)] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr133, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr134, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr141, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr142, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2267,12 +2267,12 @@ Fragment 3 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 < ('1994-01-01':Date + '1 year':Interval)) } + 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) } Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_shipmode, count, sum($expr133), sum($expr134)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [lineitem_l_shipmode, count, sum($expr141), sum($expr142)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [orders_o_orderkey, orders_o_orderpriority], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 2 { columns: [orders_o_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 3 { columns: [lineitem_l_orderkey, lineitem_l_shipmode, lineitem_l_linenumber], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } @@ -2332,7 +2332,7 @@ └─BatchFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], distribution: UpstreamHashShard(orders.o_orderkey) } stream_plan: | - StreamMaterialize { columns: [c_count, custdist], pk_columns: [c_count], order_descs: [custdist, c_count] } + StreamMaterialize { columns: [c_count, custdist], pk_columns: [c_count], order_descs: [custdist, c_count], pk_conflict: "no check" } └─StreamProject { exprs: [count(orders.o_orderkey), count] } └─StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count, count] } └─StreamExchange { dist: HashShard(count(orders.o_orderkey)) } @@ -2347,7 +2347,7 @@ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [c_count, custdist], pk_columns: [c_count], order_descs: [custdist, c_count] } + StreamMaterialize { columns: [c_count, custdist], pk_columns: [c_count], order_descs: [custdist, c_count], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(orders.o_orderkey), count] } StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count, count] } @@ -2419,38 +2419,38 @@ └─BatchSimpleAgg { aggs: [sum(sum($expr1)), sum(sum($expr2))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum($expr1), sum($expr2)] } - └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2] } + └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - 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] } └─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-09-01':Date + '1 mon':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [promo_revenue], pk_columns: [] } + StreamMaterialize { columns: [promo_revenue], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / sum(sum($expr2))) as $expr3] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1)), sum(sum($expr2))] } └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr1), sum($expr2)] } - └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } | └─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-09-01':Date + '1 mon':Interval)) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } | └─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) } └─StreamExchange { dist: HashShard(part.p_partkey) } └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [promo_revenue], pk_columns: [] } + StreamMaterialize { columns: [promo_revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [((100.00:Decimal * sum(sum($expr199))) / sum(sum($expr200))) as $expr202] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr199)), sum(sum($expr200))] } + StreamProject { exprs: [((100.00:Decimal * sum(sum($expr212))) / sum(sum($expr213))) as $expr214] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr212)), sum(sum($expr213))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr199), sum($expr200)] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr199, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr200, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr212), sum($expr213)] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr212, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr213, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2458,7 +2458,7 @@ 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-09-01':Date + '1 mon':Interval)) } + 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) } Upstream BatchPlanNode @@ -2468,7 +2468,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum($expr199)), sum(sum($expr200))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr212)), sum(sum($expr213))], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_type], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -2554,7 +2554,7 @@ | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1)] } | └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1] } - | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─BatchScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } └─BatchExchange { order: [], dist: HashShard(max(max(sum($expr2)))) } └─BatchSimpleAgg { aggs: [max(max(sum($expr2)))] } @@ -2563,22 +2563,22 @@ └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr2)] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr117)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr117)))] } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr121)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr121)))], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: all } ├─StreamExchange { dist: HashShard(sum($expr1)) } | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } | ├─StreamExchange { dist: HashShard(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) } | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - | └─StreamShare { id = 1033 } + | └─StreamShare { id = 1277 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr1)] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - 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-01-01':Date + '3 mons':Interval)) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─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) } └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } └─StreamProject { exprs: [max(max(sum($expr1)))] } @@ -2587,27 +2587,27 @@ └─StreamHashAgg { group_key: [$expr2], aggs: [count, max(sum($expr1))] } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as $expr2] } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - └─StreamShare { id = 1033 } + └─StreamShare { id = 1277 } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr1)] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - 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-01-01':Date + '3 mons':Interval)) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr117)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr117)))] } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr121)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr121)))], pk_conflict: "no check" } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: sum($expr117) = max(max(sum($expr117))), output: all } + StreamHashJoin { type: Inner, predicate: sum($expr121) = max(max(sum($expr121))), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([4]) from 1 StreamExchange Hash([0]) from 5 Fragment 1 - StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr117), lineitem.l_suppkey] } + StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr121), lineitem.l_suppkey] } left table: 4, right table 6, left degree table: 5, right degree table: 7, StreamExchange Hash([0]) from 2 - StreamProject { exprs: [lineitem.l_suppkey, sum($expr117)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr121)] } StreamExchange Hash([0]) from 3 Fragment 2 @@ -2616,45 +2616,45 @@ BatchPlanNode Fragment 3 - StreamProject { exprs: [lineitem.l_suppkey, sum($expr117)] } - StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr117)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr127)] } + StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr127)] } result table: 8, state tables: [] StreamExchange Hash([0]) from 4 Fragment 4 - StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr117, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr127, 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) } Upstream BatchPlanNode Fragment 5 - StreamProject { exprs: [max(max(sum($expr117)))] } - StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr117)))] } + StreamProject { exprs: [max(max(sum($expr121)))] } + StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr121)))] } result table: 10, state tables: [9] StreamExchange Single from 6 Fragment 6 - StreamHashAgg { group_key: [$expr118], aggs: [count, max(sum($expr117))] } + StreamHashAgg { group_key: [$expr128], aggs: [count, max(sum($expr121))] } result table: 12, state tables: [11] - StreamProject { exprs: [lineitem.l_suppkey, sum($expr117), Vnode(lineitem.l_suppkey) as $expr118] } - StreamProject { exprs: [lineitem.l_suppkey, sum($expr117)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr121), Vnode(lineitem.l_suppkey) as $expr128] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr121)] } StreamExchange Hash([0]) from 3 - Table 0 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr117), lineitem_l_suppkey], primary key: [$4 ASC, $0 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [4] } - Table 1 { columns: [sum($expr117), supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } - Table 2 { columns: [max(max(sum($expr117)))], primary key: [$0 ASC], value indices: [0], distribution key: [0] } - Table 3 { columns: [max(max(sum($expr117))), _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } + Table 0 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr121), lineitem_l_suppkey], primary key: [$4 ASC, $0 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [4] } + Table 1 { columns: [sum($expr121), supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } + Table 2 { columns: [max(max(sum($expr121)))], primary key: [$0 ASC], value indices: [0], distribution key: [0] } + Table 3 { columns: [max(max(sum($expr121))), _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone], primary key: [$0 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 5 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 6 { columns: [lineitem_l_suppkey, sum($expr117)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 6 { columns: [lineitem_l_suppkey, sum($expr121)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 7 { columns: [lineitem_l_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 8 { columns: [lineitem_l_suppkey, count, sum($expr117)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 9 { columns: [max(sum($expr117)), $expr118], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } - Table 10 { columns: [count, max(max(sum($expr117)))], primary key: [], value indices: [0, 1], distribution key: [] } - Table 11 { columns: [$expr118, sum($expr117), lineitem_l_suppkey], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 12 { columns: [$expr118, count, max(sum($expr117))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } - Table 4294967294 { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey, max(max(sum($expr117)))], primary key: [$0 ASC, $5 ASC, $4 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [4] } + Table 8 { columns: [lineitem_l_suppkey, count, sum($expr127)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 9 { columns: [max(sum($expr121)), $expr128], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 10 { columns: [count, max(max(sum($expr121)))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [$expr128, sum($expr121), lineitem_l_suppkey], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 12 { columns: [$expr128, count, max(sum($expr121))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 4294967294 { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey, max(max(sum($expr121)))], primary key: [$0 ASC, $5 ASC, $4 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [4] } - id: tpch_q16 before: - create_tables @@ -2721,9 +2721,9 @@ └─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: | - StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], pk_columns: [p_brand, p_type, p_size], order_descs: [supplier_cnt, p_brand, p_type, p_size] } - └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } - └─StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count, count(distinct partsupp.ps_suppkey)] } + StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], pk_columns: [p_brand, p_type, p_size], order_descs: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: "no check" } + └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(partsupp.ps_suppkey)] } + └─StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count, count(partsupp.ps_suppkey)] } └─StreamExchange { dist: HashShard(part.p_brand, part.p_type, part.p_size) } └─StreamHashJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey, partsupp.ps_partkey, part.p_partkey] } ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } @@ -2739,7 +2739,7 @@ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], pk_columns: [p_brand, p_type, p_size], order_descs: [supplier_cnt, p_brand, p_type, p_size] } + StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], pk_columns: [p_brand, p_type, p_size], order_descs: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count, count(distinct partsupp.ps_suppkey)] } @@ -2858,7 +2858,7 @@ └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [avg_yearly], pk_columns: [] } + StreamMaterialize { columns: [avg_yearly], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr2] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(lineitem.l_extendedprice))] } └─StreamExchange { dist: Single } @@ -2888,9 +2888,9 @@ └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [avg_yearly], pk_columns: [] } + StreamMaterialize { columns: [avg_yearly], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr145] } + StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr153] } StreamGlobalSimpleAgg { aggs: [count, sum(sum(lineitem.l_extendedprice))] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -2898,11 +2898,11 @@ Fragment 1 StreamStatelessLocalSimpleAgg { aggs: [count, sum(lineitem.l_extendedprice)] } StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey, part.p_partkey] } - StreamFilter { predicate: (lineitem.l_quantity < $expr143) } + StreamFilter { predicate: (lineitem.l_quantity < $expr152) } StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([2]) from 2 - StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr143] } + StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr152] } StreamHashAgg { group_key: [part.p_partkey], aggs: [count, sum(lineitem.l_quantity), count(lineitem.l_quantity)] } result table: 9, state tables: [] StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } @@ -2947,7 +2947,7 @@ Table 0 { columns: [count, sum(sum(lineitem_l_extendedprice))], primary key: [], value indices: [0, 1], distribution key: [] } Table 1 { columns: [lineitem_l_quantity, lineitem_l_extendedprice, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey], primary key: [$2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2] } Table 2 { columns: [part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } - Table 3 { columns: [part_p_partkey, $expr143], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 3 { columns: [part_p_partkey, $expr152], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [part_p_partkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 5 { columns: [lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 6 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } @@ -3049,7 +3049,7 @@ └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], pk_columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice], order_descs: [o_totalprice, o_orderdate, c_name, c_custkey, o_orderkey] } + StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], pk_columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice], order_descs: [o_totalprice, o_orderdate, c_name, c_custkey, o_orderkey], pk_conflict: "no check" } └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } └─StreamTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0 } └─StreamExchange { dist: Single } @@ -3075,7 +3075,7 @@ └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], pk_columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice], order_descs: [o_totalprice, o_orderdate, c_name, c_custkey, o_orderkey] } + StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], pk_columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice], order_descs: [o_totalprice, o_orderdate, c_name, c_custkey, o_orderkey], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } StreamTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0 } @@ -3085,7 +3085,7 @@ Fragment 1 StreamGroupTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0, group_key: [6] } state table: 1 - StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } + StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr3] } StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [count, sum(lineitem.l_quantity)] } result table: 2, state tables: [] @@ -3128,8 +3128,8 @@ Upstream BatchPlanNode - Table 0 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [] } - Table 1 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1], primary key: [$6 ASC, $4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [2], vnode column idx: 6 } + Table 0 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr3], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [] } + Table 1 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr3], primary key: [$6 ASC, $4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [2], vnode column idx: 6 } Table 2 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, count, sum(lineitem_l_quantity)], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6], distribution key: [2] } Table 3 { columns: [customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, lineitem_l_quantity, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$2 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [2] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } @@ -3209,7 +3209,7 @@ └─BatchFilter { predicate: In(lineitem.l_shipmode, 'AIR':Varchar, 'AIR REG':Varchar) AND (lineitem.l_shipinstruct = 'DELIVER IN PERSON':Varchar) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipinstruct, lineitem.l_shipmode], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [revenue], pk_columns: [] } + StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum($expr1))] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1))] } └─StreamExchange { dist: Single } @@ -3226,16 +3226,16 @@ └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [revenue], pk_columns: [] } + StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [sum(sum($expr68))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr68))] } + StreamProject { exprs: [sum(sum($expr72))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr72))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr68)] } - StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr68, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr72)] } + StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr72, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamFilter { predicate: (((((((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:Int32)) AND (lineitem.l_quantity <= 11:Int32)) 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:Int32)) AND (lineitem.l_quantity <= 40:Int32)) 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:Int32)) AND (lineitem.l_quantity <= 20:Int32)) AND (part.p_size <= 15:Int32))) } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, @@ -3255,7 +3255,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum($expr68))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr72))], primary key: [], value indices: [0, 1], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_brand, part_p_size, part_p_container], primary key: [$0 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -3363,10 +3363,10 @@ | └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity] } - └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], pk_columns: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], order_descs: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], pk_columns: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], order_descs: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], pk_conflict: "no check" } └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } @@ -3398,11 +3398,11 @@ | └─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) } └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], pk_columns: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], order_descs: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], pk_columns: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], order_descs: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], pk_conflict: "no check" } materialized table: 4294967294 StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -3429,11 +3429,11 @@ Fragment 4 StreamProject { exprs: [partsupp.ps_suppkey, partsupp.ps_partkey, partsupp.ps_partkey, partsupp.ps_suppkey] } - StreamFilter { predicate: ($expr108 > $expr107) } + StreamFilter { predicate: ($expr115 > $expr116) } StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM partsupp.ps_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM partsupp.ps_suppkey, output: all } left table: 8, right table 10, left degree table: 9, right degree table: 11, StreamExchange Hash([0, 1]) from 5 - StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr107] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr116] } StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [count, sum(lineitem.l_quantity)] } result table: 16, state tables: [] StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey] } @@ -3442,7 +3442,7 @@ StreamExchange Hash([0, 1]) from 9 Fragment 5 - StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr108] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr115] } StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: all } left table: 12, right table 14, left degree table: 13, right degree table: 15, StreamExchange Hash([0]) from 6 @@ -3470,7 +3470,7 @@ Fragment 9 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (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) } Upstream BatchPlanNode @@ -3483,9 +3483,9 @@ Table 5 { columns: [supplier_s_nationkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 6 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 7 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 8 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr108], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 8 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr115], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } Table 9 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } - Table 10 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr107], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 10 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr116], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } Table 11 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } Table 12 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_availqty], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 13 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } @@ -3604,7 +3604,7 @@ └─BatchFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [s_name, numwait], pk_columns: [s_name], order_descs: [numwait, s_name] } + StreamMaterialize { columns: [s_name, numwait], pk_columns: [s_name], order_descs: [numwait, s_name], pk_conflict: "no check" } └─StreamProject { exprs: [supplier.s_name, count] } └─StreamTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0 } └─StreamExchange { dist: Single } @@ -3642,7 +3642,7 @@ └─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) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [s_name, numwait], pk_columns: [s_name], order_descs: [numwait, s_name] } + StreamMaterialize { columns: [s_name, numwait], pk_columns: [s_name], order_descs: [numwait, s_name], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [supplier.s_name, count] } StreamTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0 } @@ -3652,7 +3652,7 @@ Fragment 1 StreamGroupTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0, group_key: [2] } state table: 1 - StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr1] } + StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr3] } StreamProject { exprs: [supplier.s_name, count] } StreamHashAgg { group_key: [supplier.s_name], aggs: [count, count] } result table: 2, state tables: [] @@ -3720,8 +3720,8 @@ Upstream BatchPlanNode - Table 0 { columns: [supplier_s_name, count, $expr1], primary key: [$1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [] } - Table 1 { columns: [supplier_s_name, count, $expr1], primary key: [$2 ASC, $1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [0], vnode column idx: 2 } + Table 0 { columns: [supplier_s_name, count, $expr3], primary key: [$1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [] } + Table 1 { columns: [supplier_s_name, count, $expr3], primary key: [$2 ASC, $1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [0], vnode column idx: 2 } Table 2 { columns: [supplier_s_name, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 3 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [1] } Table 4 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } @@ -3834,7 +3834,7 @@ └─BatchFilter { predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } └─BatchScan { table: customer, columns: [customer.c_acctbal, customer.c_phone], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [cntrycode, numcust, totacctbal], pk_columns: [cntrycode] } + StreamMaterialize { columns: [cntrycode, numcust, totacctbal], pk_columns: [cntrycode], pk_conflict: "no check" } └─StreamProject { exprs: [$expr2, count, sum(customer.c_acctbal)] } └─StreamHashAgg { group_key: [$expr2], aggs: [count, count, sum(customer.c_acctbal)] } └─StreamExchange { dist: HashShard($expr2) } diff --git a/src/frontend/planner_test/tests/testdata/union.yaml b/src/frontend/planner_test/tests/testdata/union.yaml index b904226c0f06e..12ff56b04204e 100644 --- a/src/frontend/planner_test/tests/testdata/union.yaml +++ b/src/frontend/planner_test/tests/testdata/union.yaml @@ -10,7 +10,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a, b, c, t1._row_id(hidden), null:Int64(hidden), 0:Int32(hidden)], pk_columns: [t1._row_id, null:Int64, 0:Int32] } + StreamMaterialize { columns: [a, b, c, t1._row_id(hidden), null:Int64(hidden), 0:Int32(hidden)], pk_columns: [t1._row_id, null:Int64, 0:Int32], pk_conflict: "no check" } └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(t1._row_id, null:Int64, 0:Int32) } | └─StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, null:Int64, 0:Int32] } @@ -20,7 +20,7 @@ └─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a, b, c, t1._row_id(hidden), null:Int64(hidden), 0:Int32(hidden)], pk_columns: [t1._row_id, null:Int64, 0:Int32] } + StreamMaterialize { columns: [a, b, c, t1._row_id(hidden), null:Int64(hidden), 0:Int32(hidden)], pk_columns: [t1._row_id, null:Int64, 0:Int32], pk_conflict: "no check" } materialized table: 4294967294 StreamUnion { all: true } StreamExchange Hash([3, 4, 5]) from 1 @@ -58,7 +58,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [a, b, c], pk_columns: [a, b, c] } + StreamMaterialize { columns: [a, b, c], pk_columns: [a, b, c], pk_conflict: "no check" } └─StreamProject { exprs: [t1.a, t1.b, t1.c] } └─StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } └─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } @@ -71,7 +71,7 @@ └─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a, b, c], pk_columns: [a, b, c] } + StreamMaterialize { columns: [a, b, c], pk_columns: [a, b, c], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [t1.a, t1.b, t1.c] } StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } @@ -116,7 +116,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: UpstreamHashShard(t2.a) } stream_plan: | - StreamMaterialize { columns: [a, b, c], pk_columns: [a, b, c] } + StreamMaterialize { columns: [a, b, c], pk_columns: [a, b, c], pk_conflict: "no check" } └─StreamProject { exprs: [t1.a, t1.b, t1.c] } └─StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } └─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } @@ -129,7 +129,7 @@ └─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [a, b, c], pk_columns: [a, b, c] } + StreamMaterialize { columns: [a, b, c], pk_columns: [a, b, c], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [t1.a, t1.b, t1.c] } StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } diff --git a/src/frontend/planner_test/tests/testdata/watermark.yaml b/src/frontend/planner_test/tests/testdata/watermark.yaml index 041a527e31a4b..b18dfa545f184 100644 --- a/src/frontend/planner_test/tests/testdata/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/watermark.yaml @@ -7,9 +7,9 @@ LogicalProject { exprs: [(v1 - '00:00:02':Interval) as $expr1] } └─LogicalSource { source: t, columns: [v1, _row_id], time_range: [(Unbounded, Unbounded)] } stream_plan: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [(v1 - '00:00:02':Interval) as $expr1, _row_id], watermark_columns: [(v1 - '00:00:02':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id], watermark_columns: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval)] } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } └─StreamSource { source: "t", columns: ["v1", "_row_id"] } diff --git a/src/frontend/src/binder/delete.rs b/src/frontend/src/binder/delete.rs index d6811e5ff5115..09d4b2f56d154 100644 --- a/src/frontend/src/binder/delete.rs +++ b/src/frontend/src/binder/delete.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::Schema; +use risingwave_common::catalog::{Schema, TableVersionId}; use risingwave_common::error::Result; use risingwave_sqlparser::ast::{Expr, ObjectName, SelectItem}; @@ -26,6 +26,9 @@ pub struct BoundDelete { /// Id of the table to perform deleting. pub table_id: TableId, + /// Version id of the table. + pub table_version_id: TableVersionId, + /// Name of the table to perform deleting. pub table_name: String, @@ -58,12 +61,14 @@ impl Binder { let table_catalog = self.resolve_dml_table(schema_name, &table_name, false)?; let table_id = table_catalog.id; let owner = table_catalog.owner; + let table_version_id = table_catalog.version_id().expect("table must be versioned"); let table = self.bind_table(schema_name, &table_name, None)?; let (returning_list, fields) = self.bind_returning_list(returning_items)?; let returning = !returning_list.is_empty(); let delete = BoundDelete { table_id, + table_version_id, table_name, owner, table, diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 2a77882f30658..8969e2ac4a201 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -375,6 +375,13 @@ impl Binder { ("array_join", raw_call(ExprType::ArrayToString)), ("array_prepend", raw_call(ExprType::ArrayPrepend)), ("array_to_string", raw_call(ExprType::ArrayToString)), + // jsonb + ("jsonb_object_field", raw_call(ExprType::JsonbAccessInner)), + ("jsonb_array_element", raw_call(ExprType::JsonbAccessInner)), + ("jsonb_object_field_text", raw_call(ExprType::JsonbAccessStr)), + ("jsonb_array_element_text", raw_call(ExprType::JsonbAccessStr)), + ("jsonb_typeof", raw_call(ExprType::JsonbTypeof)), + ("jsonb_array_length", raw_call(ExprType::JsonbArrayLength)), // System information operations. ( "pg_typeof", diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index de782f7e02110..2f315f6491673 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -40,6 +40,7 @@ impl Binder { Expr::TypedString { data_type, value } => { let s: ExprImpl = self.bind_string(value)?.into(); s.cast_explicit(bind_data_type(&data_type)?) + .map_err(Into::into) } Expr::Row(exprs) => self.bind_row(exprs), // input ref @@ -430,7 +431,7 @@ impl Binder { return self.bind_array_cast(expr.clone(), data_type); } let lhs = self.bind_expr(expr)?; - lhs.cast_explicit(data_type) + lhs.cast_explicit(data_type).map_err(Into::into) } } diff --git a/src/frontend/src/binder/expr/value.rs b/src/frontend/src/binder/expr/value.rs index 514a848ee17fe..d2ec9e1daf504 100644 --- a/src/frontend/src/binder/expr/value.rs +++ b/src/frontend/src/binder/expr/value.rs @@ -130,7 +130,7 @@ impl Binder { }, ) .into(); - return lhs.cast_explicit(ty); + return lhs.cast_explicit(ty).map_err(Into::into); } let inner_type = if let DataType::List { datatype } = &ty { *datatype.clone() diff --git a/src/frontend/src/binder/insert.rs b/src/frontend/src/binder/insert.rs index 6b849bbd9b2bb..a0cfe457183d1 100644 --- a/src/frontend/src/binder/insert.rs +++ b/src/frontend/src/binder/insert.rs @@ -15,7 +15,7 @@ use std::collections::HashSet; use itertools::Itertools; -use risingwave_common::catalog::Schema; +use risingwave_common::catalog::{Schema, TableVersionId}; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; @@ -32,6 +32,9 @@ pub struct BoundInsert { /// Id of the table to perform inserting. pub table_id: TableId, + /// Version id of the table. + pub table_version_id: TableVersionId, + /// Name of the table to perform inserting. pub table_name: String, @@ -77,6 +80,7 @@ impl Binder { let table_catalog = self.resolve_dml_table(schema_name.as_deref(), &table_name, true)?; let table_id = table_catalog.id; let owner = table_catalog.owner; + let table_version_id = table_catalog.version_id().expect("table must be versioned"); let columns_to_insert = table_catalog .columns .clone() @@ -220,6 +224,7 @@ impl Binder { let insert = BoundInsert { table_id, + table_version_id, table_name, owner, row_id_index, @@ -247,7 +252,7 @@ impl Binder { return exprs .into_iter() .zip_eq_fast(expected_types) - .map(|(e, t)| e.cast_assign(t.clone())) + .map(|(e, t)| e.cast_assign(t.clone()).map_err(Into::into)) .try_collect(); } std::cmp::Ordering::Less => "INSERT has more expressions than target columns", diff --git a/src/frontend/src/binder/update.rs b/src/frontend/src/binder/update.rs index 0567b687c35bd..b1b29904157ff 100644 --- a/src/frontend/src/binder/update.rs +++ b/src/frontend/src/binder/update.rs @@ -16,7 +16,7 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; use itertools::Itertools; -use risingwave_common::catalog::Schema; +use risingwave_common::catalog::{Schema, TableVersionId}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_sqlparser::ast::{Assignment, Expr, ObjectName, SelectItem}; @@ -31,6 +31,9 @@ pub struct BoundUpdate { /// Id of the table to perform updating. pub table_id: TableId, + /// Version id of the table. + pub table_version_id: TableVersionId, + /// Name of the table to perform updating. pub table_name: String, @@ -68,6 +71,7 @@ impl Binder { let table_catalog = self.resolve_dml_table(schema_name.as_deref(), &table_name, false)?; let table_id = table_catalog.id; let owner = table_catalog.owner; + let table_version_id = table_catalog.version_id().expect("table must be versioned"); let table = self.bind_relation_by_name(name, None)?; @@ -132,6 +136,7 @@ impl Binder { Ok(BoundUpdate { table_id, + table_version_id, table_name, owner, table, diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 16f8f3e04e75b..4ffe20c8d87a6 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -327,6 +327,11 @@ impl TableCatalog { self.version.as_ref() } + /// Get the table's version id. Returns `None` if the table has no version field. + pub fn version_id(&self) -> Option { + self.version().map(|v| v.version_id) + } + pub fn to_prost(&self, schema_id: SchemaId, database_id: DatabaseId) -> ProstTable { ProstTable { id: self.id.table_id, diff --git a/src/frontend/src/expr/function_call.rs b/src/frontend/src/expr/function_call.rs index fa3d6a98d33ac..10897199c8e69 100644 --- a/src/frontend/src/expr/function_call.rs +++ b/src/frontend/src/expr/function_call.rs @@ -14,10 +14,11 @@ use itertools::Itertools; use risingwave_common::catalog::Schema; -use risingwave_common::error::{ErrorCode, Result}; +use risingwave_common::error::{ErrorCode, Result as RwResult, RwError}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::vector_op::cast::literal_parsing; +use thiserror::Error; use super::{cast_ok, infer_some_all, infer_type, CastContext, Expr, ExprImpl, Literal}; use crate::expr::{ExprDisplay, ExprType}; @@ -99,7 +100,7 @@ impl FunctionCall { // The functions listed here are all variadic. Type signatures of functions that take a fixed // number of arguments are checked // [elsewhere](crate::expr::type_inference::build_type_derive_map). - pub fn new(func_type: ExprType, mut inputs: Vec) -> Result { + pub fn new(func_type: ExprType, mut inputs: Vec) -> RwResult { let return_type = infer_type(func_type, &mut inputs)?; Ok(Self { func_type, @@ -109,12 +110,16 @@ impl FunctionCall { } /// Create a cast expr over `child` to `target` type in `allows` context. - pub fn new_cast(child: ExprImpl, target: DataType, allows: CastContext) -> Result { + pub fn new_cast( + child: ExprImpl, + target: DataType, + allows: CastContext, + ) -> Result { if is_row_function(&child) { // Row function will have empty fields in Datatype::Struct at this point. Therefore, // we will need to take some special care to generate the cast types. For normal struct // types, they will be handled in `cast_ok`. - return Self::cast_nested(child, target, allows); + return Self::cast_row_expr(child, target, allows); } if child.is_unknown() { // `is_unknown` makes sure `as_literal` and `as_utf8` will never panic. @@ -146,46 +151,51 @@ impl FunctionCall { } .into()) } else { - Err(ErrorCode::BindError(format!( + Err(CastError(format!( "cannot cast type \"{}\" to \"{}\" in {:?} context", source, target, allows - )) - .into()) + ))) } } /// Cast a `ROW` expression to the target type. We intentionally disallow casting arbitrary /// expressions, like `ROW(1)::STRUCT` to `STRUCT`, although an integer /// is castible to VARCHAR. It's to simply the casting rules. - fn cast_nested(expr: ExprImpl, target_type: DataType, allows: CastContext) -> Result { + fn cast_row_expr( + expr: ExprImpl, + target_type: DataType, + allows: CastContext, + ) -> Result { let func = *expr.into_function_call().unwrap(); let (fields, field_names) = if let DataType::Struct(t) = &target_type { (t.fields.clone(), t.field_names.clone()) } else { - return Err(ErrorCode::BindError(format!( - "column is of type '{}' but expression is of type record", - target_type - )) - .into()); + return Err(CastError(format!( + "cannot cast type \"{}\" to \"{}\" in {:?} context", + func.return_type(), + target_type, + allows + ))); }; let (func_type, inputs, _) = func.decompose(); - let msg = match fields.len().cmp(&inputs.len()) { + match fields.len().cmp(&inputs.len()) { std::cmp::Ordering::Equal => { let inputs = inputs .into_iter() .zip_eq_fast(fields.to_vec()) .map(|(e, t)| Self::new_cast(e, t, allows)) - .collect::>>()?; + .collect::, CastError>>()?; let return_type = DataType::new_struct( inputs.iter().map(|i| i.return_type()).collect_vec(), field_names, ); - return Ok(FunctionCall::new_unchecked(func_type, inputs, return_type).into()); + Ok(FunctionCall::new_unchecked(func_type, inputs, return_type).into()) } - std::cmp::Ordering::Less => "Input has too few columns.", - std::cmp::Ordering::Greater => "Input has too many columns.", - }; - Err(ErrorCode::BindError(format!("cannot cast record to {} ({})", target_type, msg)).into()) + std::cmp::Ordering::Less => Err(CastError("Input has too few columns.".to_string())), + std::cmp::Ordering::Greater => { + Err(CastError("Input has too many columns.".to_string())) + } + } } /// Construct a `FunctionCall` expr directly with the provided `return_type`, bypassing type @@ -205,7 +215,7 @@ impl FunctionCall { pub fn new_binary_op_func( mut func_types: Vec, mut inputs: Vec, - ) -> Result { + ) -> RwResult { let expr_type = func_types.remove(0); match expr_type { ExprType::Some | ExprType::All => { @@ -274,7 +284,7 @@ impl FunctionCall { function_call: &risingwave_pb::expr::FunctionCall, expr_type: ExprType, ret_type: DataType, - ) -> Result { + ) -> RwResult { let inputs: Vec<_> = function_call .get_children() .iter() @@ -419,3 +429,19 @@ pub fn is_row_function(expr: &ExprImpl) -> bool { } false } + +#[derive(Debug, Error)] +#[error("{0}")] +pub struct CastError(String); + +impl From for ErrorCode { + fn from(value: CastError) -> Self { + ErrorCode::BindError(value.to_string()) + } +} + +impl From for RwError { + fn from(value: CastError) -> Self { + ErrorCode::from(value).into() + } +} diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index 427c1565fa0eb..cd217b7eab50b 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -16,7 +16,7 @@ use enum_as_inner::EnumAsInner; use fixedbitset::FixedBitSet; use paste::paste; use risingwave_common::array::ListValue; -use risingwave_common::error::Result; +use risingwave_common::error::Result as RwResult; use risingwave_common::types::{DataType, Datum, Scalar}; use risingwave_expr::expr::{build_from_prost, AggKind}; use risingwave_pb::expr::expr_node::RexNode; @@ -177,22 +177,22 @@ impl ExprImpl { } /// Shorthand to create cast expr to `target` type in implicit context. - pub fn cast_implicit(self, target: DataType) -> Result { + pub fn cast_implicit(self, target: DataType) -> Result { FunctionCall::new_cast(self, target, CastContext::Implicit) } /// Shorthand to create cast expr to `target` type in assign context. - pub fn cast_assign(self, target: DataType) -> Result { + pub fn cast_assign(self, target: DataType) -> Result { FunctionCall::new_cast(self, target, CastContext::Assign) } /// Shorthand to create cast expr to `target` type in explicit context. - pub fn cast_explicit(self, target: DataType) -> Result { + pub fn cast_explicit(self, target: DataType) -> Result { FunctionCall::new_cast(self, target, CastContext::Explicit) } /// Shorthand to enforce implicit cast to boolean - pub fn enforce_bool_clause(self, clause: &str) -> Result { + pub fn enforce_bool_clause(self, clause: &str) -> RwResult { if self.is_unknown() { let inner = self.cast_implicit(DataType::Boolean)?; return Ok(inner); @@ -218,26 +218,27 @@ impl ExprImpl { /// References in `PostgreSQL`: /// * [cast](https://github.com/postgres/postgres/blob/a3ff08e0b08dbfeb777ccfa8f13ebaa95d064c04/src/include/catalog/pg_cast.dat#L437-L444) /// * [impl](https://github.com/postgres/postgres/blob/27b77ecf9f4d5be211900eda54d8155ada50d696/src/backend/utils/adt/bool.c#L204-L209) - pub fn cast_output(self) -> Result { + pub fn cast_output(self) -> RwResult { if self.return_type() == DataType::Boolean { return Ok(FunctionCall::new(ExprType::BoolOut, vec![self])?.into()); } // Use normal cast for other types. Both `assign` and `explicit` can pass the castability // check and there is no difference. self.cast_assign(DataType::Varchar) + .map_err(|err| err.into()) } /// Evaluate the expression on the given input. /// /// TODO: This is a naive implementation. We should avoid proto ser/de. /// Tracking issue: - fn eval_row(&self, input: &OwnedRow) -> Result { + fn eval_row(&self, input: &OwnedRow) -> RwResult { let backend_expr = build_from_prost(&self.to_expr_proto())?; backend_expr.eval_row(input).map_err(Into::into) } /// Evaluate a constant expression. - pub fn eval_row_const(&self) -> Result { + pub fn eval_row_const(&self) -> RwResult { assert!(self.is_const()); self.eval_row(&OwnedRow::empty()) } @@ -728,7 +729,7 @@ impl ExprImpl { } } - pub fn from_expr_proto(proto: &ExprNode) -> Result { + pub fn from_expr_proto(proto: &ExprNode) -> RwResult { let rex_node = proto.get_rex_node()?; let ret_type = proto.get_return_type()?.into(); let expr_type = proto.get_expr_type()?; @@ -892,6 +893,7 @@ use risingwave_common::bail; use risingwave_common::catalog::Schema; use risingwave_common::row::OwnedRow; +use self::function_call::CastError; use crate::binder::BoundSetExpr; use crate::utils::Condition; diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index f85a3989bd8c9..d84cb5e6087bb 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -20,7 +20,7 @@ use risingwave_common::types::{unnested_list_type, DataType, ScalarImpl}; use risingwave_pb::expr::table_function::Type; use risingwave_pb::expr::TableFunction as TableFunctionProst; -use super::{Expr, ExprImpl, ExprRewriter, Result}; +use super::{Expr, ExprImpl, ExprRewriter, RwResult}; /// A table function takes a row as input and returns a table. It is also known as Set-Returning /// Function. @@ -85,7 +85,7 @@ impl FromStr for TableFunctionType { impl TableFunction { /// Create a `TableFunction` expr with the return type inferred from `func_type` and types of /// `inputs`. - pub fn new(func_type: TableFunctionType, args: Vec) -> Result { + pub fn new(func_type: TableFunctionType, args: Vec) -> RwResult { // TODO: refactor into sth like FunctionCall::new. // Current implementation is copied from legacy code. @@ -94,7 +94,7 @@ impl TableFunction { // generate_series ( start timestamp, stop timestamp, step interval ) or // generate_series ( start i32, stop i32, step i32 ) - fn type_check(exprs: &[ExprImpl]) -> Result { + fn type_check(exprs: &[ExprImpl]) -> RwResult { let mut exprs = exprs.iter(); let (start, stop, step) = exprs.next_tuple().unwrap(); match (start.return_type(), stop.return_type(), step.return_type()) { diff --git a/src/frontend/src/expr/type_inference/cast.rs b/src/frontend/src/expr/type_inference/cast.rs index 549f5b275b825..45901c41824ae 100644 --- a/src/frontend/src/expr/type_inference/cast.rs +++ b/src/frontend/src/expr/type_inference/cast.rs @@ -115,9 +115,11 @@ pub fn align_array_and_element( .enumerate() .map(|(idx, input)| { if idx == array_idx { - input.cast_implicit(array_type.clone()) + input.cast_implicit(array_type.clone()).map_err(Into::into) } else { - input.cast_implicit(common_ele_type.clone()) + input + .cast_implicit(common_ele_type.clone()) + .map_err(Into::into) } }) .try_collect(); diff --git a/src/frontend/src/expr/type_inference/func.rs b/src/frontend/src/expr/type_inference/func.rs index e7b03a64affcd..685fc65616f43 100644 --- a/src/frontend/src/expr/type_inference/func.rs +++ b/src/frontend/src/expr/type_inference/func.rs @@ -14,7 +14,7 @@ use itertools::Itertools as _; use num_integer::Integer as _; -use risingwave_common::error::{ErrorCode, Result}; +use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::struct_type::StructType; use risingwave_common::types::{DataType, DataTypeName, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; @@ -48,7 +48,7 @@ pub fn infer_type(func_type: ExprType, inputs: &mut Vec) -> Result) -> Result()?; Ok(sig.ret_type.into()) } @@ -318,7 +318,7 @@ fn infer_type_for_special( .enumerate() .map(|(i, input)| match i { // 0-th arg must be string - 0 => input.cast_implicit(DataType::Varchar), + 0 => input.cast_implicit(DataType::Varchar).map_err(Into::into), // subsequent can be any type, using the output format _ => input.cast_output(), }) diff --git a/src/frontend/src/expr/window_function.rs b/src/frontend/src/expr/window_function.rs index 7303a15557ede..351c552722e35 100644 --- a/src/frontend/src/expr/window_function.rs +++ b/src/frontend/src/expr/window_function.rs @@ -19,7 +19,7 @@ use parse_display::Display; use risingwave_common::error::ErrorCode; use risingwave_common::types::DataType; -use super::{Expr, ExprImpl, OrderBy, Result}; +use super::{Expr, ExprImpl, OrderBy, RwResult}; /// A window function performs a calculation across a set of table rows that are somehow related to /// the current row, according to the window spec `OVER (PARTITION BY .. ORDER BY ..)`. @@ -79,7 +79,7 @@ impl WindowFunction { partition_by: Vec, order_by: OrderBy, args: Vec, - ) -> Result { + ) -> RwResult { if !args.is_empty() { return Err(ErrorCode::BindError(format!( "the length of args of {function_type} function should be 0" diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 2afac6852fb8a..51b7faefbf6f6 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -176,7 +176,7 @@ pub mod tests { let sql = r#"CREATE SINK snk1 FROM mv1 WITH (connector = 'mysql', mysql.endpoint = '127.0.0.1:3306', mysql.table = '', mysql.database = '', mysql.user = '', - mysql.password = '');"#.to_string(); + mysql.password = '', format = 'append_only', force_append_only = 'true');"#.to_string(); frontend.run_sql(sql).await.unwrap(); let session = frontend.session_ref(); diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 91454ac34f7cc..ec6894bfd7f66 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -56,6 +56,8 @@ async fn extract_avro_table_schema( with_properties, schema.row_schema_location.0.as_str(), schema.use_schema_registry, + false, + None, ) .await?; let vec_column_desc = conf.map_to_columns()?; @@ -68,6 +70,47 @@ async fn extract_avro_table_schema( .collect_vec()) } +/// Map an Avro schema to a relational schema. And extract primary key columns. +async fn extract_upsert_avro_table_schema( + schema: &AvroSchema, + with_properties: &HashMap, +) -> Result<(Vec, Vec)> { + let conf = AvroParserConfig::new( + with_properties, + schema.row_schema_location.0.as_str(), + schema.use_schema_registry, + true, + None, + ) + .await?; + let vec_column_desc = conf.map_to_columns()?; + let vec_pk_desc = conf.extract_pks()?; + let pks = vec_pk_desc + .into_iter() + .map(|desc| { + vec_column_desc + .iter() + .find(|x| x.name == desc.name) + .ok_or_else(|| { + RwError::from(ErrorCode::InternalError(format!( + "Can not found primary key column {} in value schema", + desc.name + ))) + }) + }) + .map_ok(|desc| ColumnId::new(desc.column_id)) + .collect::>>()?; + Ok(( + vec_column_desc + .into_iter() + .map(|col| ColumnCatalog { + column_desc: col.into(), + is_hidden: false, + }) + .collect_vec(), + pks, + )) +} async fn extract_debezium_avro_table_pk_columns( schema: &DebeziumAvroSchema, with_properties: &HashMap, @@ -209,6 +252,7 @@ pub(crate) async fn resolve_source_schema( ); StreamSourceInfo { row_format: RowFormatType::Avro as i32, + row_schema_location: avro_schema.row_schema_location.0.clone(), use_schema_registry: avro_schema.use_schema_registry, proto_message_name: "".to_owned(), @@ -216,10 +260,60 @@ pub(crate) async fn resolve_source_schema( } } + SourceSchema::UpsertAvro(avro_schema) => { + let mut upsert_avro_primary_key = Default::default(); + if row_id_index.is_none() { + // user specify pk(s) + if columns.len() != pk_column_ids.len() || pk_column_ids.len() != 1 { + return Err(RwError::from(ProtocolError( + "You can specify single primary key column or leave the columns and primary key fields empty.".to_string(), + ))); + } + let pk_name = columns[0].column_desc.name.clone(); + + *columns = extract_avro_table_schema(avro_schema, with_properties).await?; + let pk_col = columns + .iter() + .find(|col| col.column_desc.name == pk_name) + .ok_or_else(|| { + RwError::from(ProtocolError(format!( + "Primary key {pk_name} is not found." + ))) + })?; + + *pk_column_ids = vec![pk_col.column_id()]; + upsert_avro_primary_key = pk_name; + } else { + // contains row_id, user specify some columns without pk + if !(*pk_column_ids == vec![ROW_ID_COLUMN_ID] && columns.len() == 1) { + return Err(RwError::from(ProtocolError( + "UPSERT AVRO will automatically extract the schema from the Avro schema. You can specify single primary key column or leave the columns and primary key fields empty.".to_string(), + ))); + } + let (columns_extracted, pks_extracted) = + extract_upsert_avro_table_schema(avro_schema, with_properties).await?; + + *columns = columns_extracted; + *pk_column_ids = pks_extracted; + } + + StreamSourceInfo { + row_format: RowFormatType::UpsertAvro as i32, + row_schema_location: avro_schema.row_schema_location.0.clone(), + use_schema_registry: avro_schema.use_schema_registry, + upsert_avro_primary_key, + ..Default::default() + } + } + SourceSchema::Json => StreamSourceInfo { row_format: RowFormatType::Json as i32, ..Default::default() }, + SourceSchema::UpsertJson => StreamSourceInfo { + row_format: RowFormatType::UpsertJson as i32, + ..Default::default() + }, SourceSchema::Maxwell => { // return err if user has not specified a pk diff --git a/src/frontend/src/handler/create_table_as.rs b/src/frontend/src/handler/create_table_as.rs index fb53d539b34b5..4ab605c788840 100644 --- a/src/frontend/src/handler/create_table_as.rs +++ b/src/frontend/src/handler/create_table_as.rs @@ -50,13 +50,13 @@ pub async fn handle_create_as( } } + let mut col_id_gen = ColumnIdGenerator::new_initial(); + // Generate catalog descs from query let mut column_descs: Vec<_> = { let mut binder = Binder::new(&session); let bound = binder.bind(Statement::Query(query.clone()))?; if let BoundStatement::Query(query) = bound { - let mut col_id_gen = ColumnIdGenerator::new_initial(); - // Create ColumnCatelog by Field query .schema() @@ -93,7 +93,7 @@ pub async fn handle_create_as( None, vec![], "".to_owned(), // TODO: support `SHOW CREATE TABLE` for `CREATE TABLE AS` - None, // TODO: support `ALTER TABLE` for `CREATE TABLE AS` + Some(col_id_gen.into_version()), )?; let mut graph = build_graph(plan); graph.parallelism = session diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 25497bc08c215..67cc3f3585bdb 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -29,6 +29,7 @@ use crate::binder::{Binder, Relation}; use crate::catalog::{CatalogError, IndexCatalog}; use crate::handler::util::col_descs_to_rows; use crate::handler::HandlerArgs; +use crate::optimizer::property::Direction; pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Result { let session = handler_args.session; @@ -113,11 +114,18 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res rows.extend(indices.iter().map(|index| { let index_table = index.index_table.clone(); - let index_columns = index_table + let index_columns_with_ordering = index_table .pk .iter() .filter(|x| !index_table.columns[x.index].is_hidden) - .map(|x| index_table.columns[x.index].name().to_string()) + .map(|x| { + let index_column_name = index_table.columns[x.index].name().to_string(); + if Direction::Desc == x.direct { + index_column_name + " DESC" + } else { + index_column_name + } + }) .collect_vec(); let pk_column_index_set = index_table @@ -147,7 +155,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res Some( format!( "index({}) distributed by({})", - display_comma_separated(&index_columns), + display_comma_separated(&index_columns_with_ordering), display_comma_separated(&distributed_by_columns), ) .into(), @@ -156,7 +164,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res Some( format!( "index({}) include({}) distributed by({})", - display_comma_separated(&index_columns), + display_comma_separated(&index_columns_with_ordering), display_comma_separated(&include_columns), display_comma_separated(&distributed_by_columns), ) @@ -204,7 +212,7 @@ mod tests { .unwrap(); frontend - .run_sql("create index idx1 on t (v1,v2);") + .run_sql("create index idx1 on t (v1 DESC, v2);") .await .unwrap(); @@ -233,7 +241,7 @@ mod tests { "v3".into() => "Int32".into(), "v4".into() => "Int32".into(), "primary key".into() => "v3".into(), - "idx1".into() => "index(v1, v2, v3) include(v4) distributed by(v1, v2)".into(), + "idx1".into() => "index(v1 DESC, v2, v3) include(v4) distributed by(v1, v2)".into(), }; assert_eq!(columns, expected_columns); diff --git a/src/frontend/src/handler/drop_sink.rs b/src/frontend/src/handler/drop_sink.rs index 81af96449991a..6b3a472f5cb49 100644 --- a/src/frontend/src/handler/drop_sink.rs +++ b/src/frontend/src/handler/drop_sink.rs @@ -70,7 +70,7 @@ mod tests { #[tokio::test] async fn test_drop_sink_handler() { - let sql_create_table = "create table t (v1 smallint);"; + let sql_create_table = "create table t (v1 smallint primary key);"; let sql_create_mv = "create materialized view mv as select v1 from t;"; let sql_create_sink = "create sink snk from mv with( connector = 'mysql')"; let sql_drop_sink = "drop sink snk;"; diff --git a/src/frontend/src/meta_client.rs b/src/frontend/src/meta_client.rs index a7ac0de32f44a..9fcb0f34845ac 100644 --- a/src/frontend/src/meta_client.rs +++ b/src/frontend/src/meta_client.rs @@ -14,13 +14,14 @@ use std::collections::HashMap; +use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_pb::backup_service::MetaSnapshotMetadata; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockSnapshot; use risingwave_pb::meta::list_table_fragments_response::TableFragmentInfo; use risingwave_pb::meta::CreatingJobInfo; use risingwave_rpc_client::error::Result; -use risingwave_rpc_client::{HummockMetaClient, MetaClient, SystemParamsReader}; +use risingwave_rpc_client::{HummockMetaClient, MetaClient}; /// A wrapper around the `MetaClient` that only provides a minor set of meta rpc. /// Most of the rpc to meta are delegated by other separate structs like `CatalogWriter`, diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 60d9a26286bde..0a26461ac3993 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -11,6 +11,7 @@ // 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::ops::DerefMut; pub mod plan_node; pub use plan_node::{Explain, PlanRef}; @@ -37,8 +38,8 @@ use risingwave_common::util::iter_util::ZipEqDebug; use self::heuristic_optimizer::{ApplyOrder, HeuristicOptimizer}; use self::plan_node::{ - BatchProject, Convention, LogicalProject, StreamDml, StreamMaterialize, StreamRowIdGen, - StreamSink, + BatchProject, Convention, LogicalProject, StreamDml, StreamMaterialize, StreamProject, + StreamRowIdGen, StreamSink, }; #[cfg(debug_assertions)] use self::plan_visitor::InputRefValidator; @@ -49,6 +50,7 @@ use self::plan_visitor::{ use self::property::RequiredDist; use self::rule::*; use crate::catalog::table_catalog::{TableType, TableVersion}; +use crate::expr::InputRef; use crate::optimizer::plan_node::{ BatchExchange, ColumnPruningContext, PlanNodeType, PlanTreeNode, PredicatePushdownContext, RewriteExprsRecursive, @@ -488,15 +490,22 @@ impl PlanRoot { // Convert to physical plan node plan = plan.to_batch_with_order_required(&self.required_order)?; - // TODO: SessionTimezone substitution + let ctx = plan.ctx(); + // Inline session timezone + plan = inline_session_timezone_in_exprs(ctx.clone(), plan)?; + + if ctx.is_explain_trace() { + ctx.trace("Inline Session Timezone:"); + ctx.trace(plan.explain_to_string().unwrap()); + } + // Const eval of exprs at the last minute - // plan = const_eval_exprs(plan)?; + plan = const_eval_exprs(plan)?; - // let ctx = plan.ctx(); - // if ctx.is_explain_trace() { - // ctx.trace("Const eval exprs:"); - // ctx.trace(plan.explain_to_string().unwrap()); - // } + if ctx.is_explain_trace() { + ctx.trace("Const eval exprs:"); + ctx.trace(plan.explain_to_string().unwrap()); + } #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -650,13 +659,21 @@ impl PlanRoot { ); } + // Inline session timezone + plan = inline_session_timezone_in_exprs(ctx.clone(), plan)?; + + if ctx.is_explain_trace() { + ctx.trace("Inline session timezone:"); + ctx.trace(plan.explain_to_string().unwrap()); + } + // Const eval of exprs at the last minute - // plan = const_eval_exprs(plan)?; + plan = const_eval_exprs(plan)?; - // if ctx.is_explain_trace() { - // ctx.trace("Const eval exprs:"); - // ctx.trace(plan.explain_to_string().unwrap()); - // } + if ctx.is_explain_trace() { + ctx.trace("Const eval exprs:"); + ctx.trace(plan.explain_to_string().unwrap()); + } #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -749,7 +766,24 @@ impl PlanRoot { definition: String, properties: WithOptions, ) -> Result { - let stream_plan = self.gen_stream_plan()?; + let mut stream_plan = self.gen_stream_plan()?; + + // Add a project node if there is hidden column(s). + let input_fields = stream_plan.schema().fields(); + if input_fields.len() != self.out_fields.count_ones(..) { + let exprs = input_fields + .iter() + .enumerate() + .filter_map(|(idx, field)| { + if self.out_fields.contains(idx) { + Some(InputRef::new(idx, field.data_type.clone()).into()) + } else { + None + } + }) + .collect_vec(); + stream_plan = StreamProject::new(LogicalProject::new(stream_plan, exprs)).into(); + } StreamSink::create( stream_plan, @@ -769,7 +803,6 @@ impl PlanRoot { } } -#[allow(dead_code)] fn const_eval_exprs(plan: PlanRef) -> Result { let mut const_eval_rewriter = ConstEvalRewriter { error: None }; @@ -780,6 +813,11 @@ fn const_eval_exprs(plan: PlanRef) -> Result { Ok(plan) } +fn inline_session_timezone_in_exprs(ctx: OptimizerContextRef, plan: PlanRef) -> Result { + let plan = plan.rewrite_exprs_recursive(ctx.session_timezone().deref_mut()); + Ok(plan) +} + #[cfg(test)] mod tests { use risingwave_common::catalog::Field; diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index 90f00a6054e2b..481dc87b07f21 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -14,13 +14,13 @@ use core::convert::Into; use core::fmt::Formatter; -use std::cell::RefCell; +use std::cell::{RefCell, RefMut}; use std::rc::Rc; use std::sync::Arc; use risingwave_sqlparser::ast::{ExplainOptions, ExplainType}; -use crate::expr::{CorrelatedId, ExprImpl, ExprRewriter, SessionTimezone}; +use crate::expr::{CorrelatedId, SessionTimezone}; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::PlanNodeId; use crate::session::SessionImpl; @@ -140,7 +140,9 @@ impl OptimizerContext { return; } let mut optimizer_trace = self.optimizer_trace.borrow_mut(); - optimizer_trace.push(str.into()); + let string = str.into(); + tracing::trace!("{}", string); + optimizer_trace.push(string); optimizer_trace.push("\n".to_string()); } @@ -174,9 +176,8 @@ impl OptimizerContext { &self.normalized_sql } - pub fn expr_with_session_timezone(&self, expr: ExprImpl) -> ExprImpl { - let mut session_timezone = self.session_timezone.borrow_mut(); - session_timezone.rewrite_expr(expr) + pub fn session_timezone(&self) -> RefMut<'_, SessionTimezone> { + self.session_timezone.borrow_mut() } /// Appends any information that the optimizer needs to alert the user about to the PG NOTICE diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 7e697c1fb6879..7f112abe5164d 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -14,7 +14,6 @@ use std::fmt; -use risingwave_common::catalog::INITIAL_TABLE_VERSION_ID; use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::DeleteNode; @@ -76,7 +75,7 @@ impl ToBatchProst for BatchDelete { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::Delete(DeleteNode { table_id: self.logical.table_id().table_id(), - table_version_id: INITIAL_TABLE_VERSION_ID, // TODO: use correct version id + table_version_id: self.logical.table_version_id(), returning: self.logical.has_returning(), }) } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index c92634c2713e1..9099b7712e1b9 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -18,7 +18,6 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::FilterNode; -use super::generic::GenericPlanRef; use super::{ ExprRewritable, LogicalFilter, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, }; @@ -80,10 +79,7 @@ impl ToBatchProst for BatchFilter { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::Filter(FilterNode { search_condition: Some( - self.base - .ctx() - .expr_with_session_timezone(ExprImpl::from(self.logical.predicate().clone())) - .to_expr_proto(), + ExprImpl::from(self.logical.predicate().clone()).to_expr_proto(), ), }) } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index 55cebf0190072..4728f3d0247de 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -122,7 +122,7 @@ impl ToBatchProst for BatchHashAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(PlanAggCall::to_protobuf) .collect(), group_key: self .group_key() diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index 77fdef9ffefeb..6bb3a1df42648 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -236,12 +236,7 @@ impl ToBatchProst for BatchHashJoin { .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) - .to_expr_proto() - }), + .map(|x| x.to_expr_proto()), output_indices: self .logical .output_indices() diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index 7668aa91a2b5b..3158c727a5739 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -14,7 +14,6 @@ use std::fmt; -use risingwave_common::catalog::INITIAL_TABLE_VERSION_ID; use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::InsertNode; @@ -83,7 +82,7 @@ impl ToBatchProst for BatchInsert { .collect(); NodeBody::Insert(InsertNode { table_id: self.logical.table_id().table_id(), - table_version_id: INITIAL_TABLE_VERSION_ID, // TODO: use correct version id + table_version_id: self.logical.table_version_id(), column_indices, row_id_index: self .logical 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 3a2b7a124cf99..640afea315d02 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -199,12 +199,7 @@ impl ToBatchProst for BatchLookupJoin { .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) - .to_expr_proto() - }), + .map(|x| x.to_expr_proto()), outer_side_key: self .eq_join_predicate .left_eq_indexes() @@ -239,12 +234,7 @@ impl ToBatchProst for BatchLookupJoin { .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) - .to_expr_proto() - }), + .map(|x| x.to_expr_proto()), outer_side_key: self .eq_join_predicate .left_eq_indexes() diff --git a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs index 490cc65d733c8..5f4d92021570a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs @@ -131,12 +131,7 @@ impl ToBatchProst for BatchNestedLoopJoin { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::NestedLoopJoin(NestedLoopJoinNode { join_type: self.logical.join_type() as i32, - join_cond: Some( - self.base - .ctx() - .expr_with_session_timezone(ExprImpl::from(self.logical.on().clone())) - .to_expr_proto(), - ), + join_cond: Some(ExprImpl::from(self.logical.on().clone()).to_expr_proto()), output_indices: self .logical .output_indices() diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 538575bd2f903..9dd674374fdf0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -19,7 +19,6 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectNode; use risingwave_pb::expr::ExprNode; -use super::generic::GenericPlanRef; use super::{ ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, @@ -89,12 +88,7 @@ impl ToBatchProst for BatchProject { .logical .exprs() .iter() - .map(|expr| { - self.base - .ctx() - .expr_with_session_timezone(expr.clone()) - .to_expr_proto() - }) + .map(|expr| expr.to_expr_proto()) .collect::>(); NodeBody::Project(ProjectNode { select_list }) } diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index afcb5b62a72ab..938154b897aed 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; -use super::generic::{GenericPlanRef, PlanAggCall}; +use super::generic::PlanAggCall; use super::{ ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, @@ -114,7 +114,7 @@ impl ToBatchProst for BatchSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(PlanAggCall::to_protobuf) .collect(), // We treat simple agg as a special sort agg without group key. group_key: vec![], diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index 1e3056dff4427..a6b9bd5b6b07a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -119,18 +119,13 @@ impl ToBatchProst for BatchSortAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(PlanAggCall::to_protobuf) .collect(), group_key: self .group_key() .iter() .map(|idx| ExprImpl::InputRef(Box::new(InputRef::new(*idx, DataType::Int32)))) - .map(|expr| { - self.base - .ctx() - .expr_with_session_timezone(expr) - .to_expr_proto() - }) + .map(|expr| expr.to_expr_proto()) .collect::>(), }) } diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index 94bc258e69de3..c8e6b84a82419 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -14,12 +14,10 @@ use std::fmt; -use risingwave_common::catalog::INITIAL_TABLE_VERSION_ID; use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UpdateNode; -use super::generic::GenericPlanRef; use super::{ ExprRewritable, LogicalUpdate, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, @@ -80,18 +78,13 @@ impl ToBatchProst for BatchUpdate { .logical .exprs() .iter() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x.clone()) - .to_expr_proto() - }) + .map(|x| x.to_expr_proto()) .collect(); NodeBody::Update(UpdateNode { exprs, table_id: self.logical.table_id().table_id(), - table_version_id: INITIAL_TABLE_VERSION_ID, // TODO: use correct version id + table_version_id: self.logical.table_version_id(), returning: self.logical.has_returning(), }) } diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 21b719d3821eb..0cffc31e12c96 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -19,7 +19,6 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::values_node::ExprTuple; use risingwave_pb::batch_plan::ValuesNode; -use super::generic::GenericPlanRef; use super::{ ExprRewritable, LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, ToDistributedBatch, @@ -55,15 +54,7 @@ impl BatchValues { } fn row_to_protobuf(&self, row: &[ExprImpl]) -> ExprTuple { - let cells = row - .iter() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x.clone()) - .to_expr_proto() - }) - .collect(); + let cells = row.iter().map(|x| x.to_expr_proto()).collect(); ExprTuple { cells } } } diff --git a/src/frontend/src/optimizer/plan_node/derive.rs b/src/frontend/src/optimizer/plan_node/derive.rs index 1498d2183dcb3..3a0e198f56eaa 100644 --- a/src/frontend/src/optimizer/plan_node/derive.rs +++ b/src/frontend/src/optimizer/plan_node/derive.rs @@ -23,7 +23,7 @@ use super::PlanRef; use crate::optimizer::property::{Direction, FieldOrder, Order}; pub(crate) fn derive_columns( - schema: &Schema, + input_schema: &Schema, out_names: Vec, user_cols: &FixedBitSet, ) -> Result> { @@ -39,7 +39,7 @@ pub(crate) fn derive_columns( } let mut out_name_iter = out_names.into_iter(); - let columns = schema + let columns = input_schema .fields() .iter() .enumerate() diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 7c6815107e3fe..8b83c129e54d3 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -606,7 +606,7 @@ impl PlanAggCall { }); } - pub fn to_protobuf(&self, ctx: OptimizerContextRef) -> ProstAggCall { + pub fn to_protobuf(&self) -> ProstAggCall { ProstAggCall { r#type: self.agg_kind.to_prost().into(), return_type: Some(self.return_type.to_protobuf()), @@ -617,10 +617,7 @@ impl PlanAggCall { .iter() .map(PlanAggOrderByField::to_protobuf) .collect(), - filter: self - .filter - .as_expr_unless_true() - .map(|x| ctx.expr_with_session_timezone(x).to_expr_proto()), + filter: self.filter.as_expr_unless_true().map(|x| x.to_expr_proto()), } } diff --git a/src/frontend/src/optimizer/plan_node/logical_delete.rs b/src/frontend/src/optimizer/plan_node/logical_delete.rs index d69cc6c867c04..5eb72a8923b2b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_delete.rs +++ b/src/frontend/src/optimizer/plan_node/logical_delete.rs @@ -14,7 +14,7 @@ use std::{fmt, vec}; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::{Field, Schema, TableVersionId}; use risingwave_common::error::Result; use risingwave_common::types::DataType; @@ -37,13 +37,20 @@ pub struct LogicalDelete { pub base: PlanBase, table_name: String, // explain-only table_id: TableId, + table_version_id: TableVersionId, input: PlanRef, returning: bool, } impl LogicalDelete { /// Create a [`LogicalDelete`] node. Used internally by optimizer. - pub fn new(input: PlanRef, table_name: String, table_id: TableId, returning: bool) -> Self { + pub fn new( + input: PlanRef, + table_name: String, + table_id: TableId, + table_version_id: TableVersionId, + returning: bool, + ) -> Self { let ctx = input.ctx(); let schema = if returning { input.schema().clone() @@ -56,6 +63,7 @@ impl LogicalDelete { base, table_name, table_id, + table_version_id, input, returning, } @@ -66,9 +74,16 @@ impl LogicalDelete { input: PlanRef, table_name: String, table_id: TableId, + table_version_id: TableVersionId, returning: bool, ) -> Result { - Ok(Self::new(input, table_name, table_id, returning)) + Ok(Self::new( + input, + table_name, + table_id, + table_version_id, + returning, + )) } pub(super) fn fmt_with_name(&self, f: &mut fmt::Formatter<'_>, name: &str) -> fmt::Result { @@ -93,6 +108,10 @@ impl LogicalDelete { pub fn has_returning(&self) -> bool { self.returning } + + pub fn table_version_id(&self) -> TableVersionId { + self.table_version_id + } } impl PlanTreeNodeUnary for LogicalDelete { @@ -105,6 +124,7 @@ impl PlanTreeNodeUnary for LogicalDelete { input, self.table_name.clone(), self.table_id, + self.table_version_id, self.returning, ) } diff --git a/src/frontend/src/optimizer/plan_node/logical_insert.rs b/src/frontend/src/optimizer/plan_node/logical_insert.rs index e18f7b7518049..81e79ea4c193e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_insert.rs +++ b/src/frontend/src/optimizer/plan_node/logical_insert.rs @@ -14,7 +14,7 @@ use std::fmt; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::{Field, Schema, TableVersionId}; use risingwave_common::error::Result; use risingwave_common::types::DataType; @@ -38,6 +38,7 @@ pub struct LogicalInsert { pub base: PlanBase, table_name: String, // explain-only table_id: TableId, + table_version_id: TableVersionId, input: PlanRef, column_indices: Vec, // columns in which to insert row_id_index: Option, @@ -50,6 +51,7 @@ impl LogicalInsert { input: PlanRef, table_name: String, table_id: TableId, + table_version_id: TableVersionId, column_indices: Vec, row_id_index: Option, returning: bool, @@ -66,6 +68,7 @@ impl LogicalInsert { base, table_name, table_id, + table_version_id, input, column_indices, row_id_index, @@ -78,6 +81,7 @@ impl LogicalInsert { input: PlanRef, table_name: String, table_id: TableId, + table_version_id: TableVersionId, column_indices: Vec, row_id_index: Option, returning: bool, @@ -86,6 +90,7 @@ impl LogicalInsert { input, table_name, table_id, + table_version_id, column_indices, row_id_index, returning, @@ -125,6 +130,10 @@ impl LogicalInsert { pub fn has_returning(&self) -> bool { self.returning } + + pub fn table_version_id(&self) -> TableVersionId { + self.table_version_id + } } impl PlanTreeNodeUnary for LogicalInsert { @@ -137,6 +146,7 @@ impl PlanTreeNodeUnary for LogicalInsert { input, self.table_name.clone(), self.table_id, + self.table_version_id, self.column_indices.clone(), self.row_id_index, self.returning, diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index bfe1b19303407..219af17c0d0b9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -544,8 +544,12 @@ impl LogicalScan { let (scan, predicate, project_expr) = scan.predicate_pull_up(); if predicate.always_false() { - return LogicalValues::create(vec![], scan.schema().clone(), scan.ctx()).to_batch(); + let plan = + LogicalValues::create(vec![], scan.schema().clone(), scan.ctx()).to_batch()?; + assert_eq!(plan.schema(), self.schema()); + return required_order.enforce_if_not_satisfies(plan); } + let mut plan: PlanRef = BatchSeqScan::new(scan, scan_ranges).into(); if !predicate.always_true() { plan = BatchFilter::new(LogicalFilter::new(plan, predicate)).into(); @@ -606,17 +610,7 @@ impl ToBatch for LogicalScan { } fn to_batch_with_order_required(&self, required_order: &Order) -> Result { - // rewrite the condition before converting to batch as we will handle the expressions in a - // special way - let new_predicate = Condition { - conjunctions: self - .predicate() - .conjunctions - .iter() - .map(|expr| self.base.ctx().expr_with_session_timezone(expr.clone())) - .collect(), - }; - let new = self.clone_with_predicate(new_predicate); + let new = self.clone_with_predicate(self.predicate().clone()); if !new.indexes().is_empty() { let index_selection_rule = IndexSelectionRule::create(); diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 1486ae0ea72d4..5497f72588c54 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -22,7 +22,7 @@ use risingwave_common::catalog::{ColumnDesc, Schema}; use risingwave_common::error::Result; use risingwave_connector::source::DataType; -use super::generic::{GenericPlanNode, GenericPlanRef}; +use super::generic::GenericPlanNode; use super::stream_watermark_filter::StreamWatermarkFilter; use super::{ generic, BatchSource, ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, PlanBase, @@ -326,7 +326,6 @@ impl PredicatePushdown for LogicalSource { let mut new_conjunctions = Vec::with_capacity(predicate.conjunctions.len()); for expr in predicate.conjunctions { - let expr = self.base.ctx().expr_with_session_timezone(expr); if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, &self.base.schema) { // Not recognized, so push back new_conjunctions.push(e); diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index 932d9a02fea9d..5f4702b565c54 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -14,7 +14,7 @@ use std::{fmt, vec}; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::{Field, Schema, TableVersionId}; use risingwave_common::error::Result; use risingwave_common::types::DataType; @@ -39,6 +39,7 @@ pub struct LogicalUpdate { pub base: PlanBase, table_name: String, // explain-only table_id: TableId, + table_version_id: TableVersionId, input: PlanRef, exprs: Vec, returning: bool, @@ -50,6 +51,7 @@ impl LogicalUpdate { input: PlanRef, table_name: String, table_id: TableId, + table_version_id: TableVersionId, exprs: Vec, returning: bool, ) -> Self { @@ -65,6 +67,7 @@ impl LogicalUpdate { base, table_name, table_id, + table_version_id, input, exprs, returning, @@ -76,10 +79,18 @@ impl LogicalUpdate { input: PlanRef, table_name: String, table_id: TableId, + table_version_id: TableVersionId, exprs: Vec, returning: bool, ) -> Result { - Ok(Self::new(input, table_name, table_id, exprs, returning)) + Ok(Self::new( + input, + table_name, + table_id, + table_version_id, + exprs, + returning, + )) } pub(super) fn fmt_with_name(&self, f: &mut fmt::Formatter<'_>, name: &str) -> fmt::Result { @@ -109,6 +120,10 @@ impl LogicalUpdate { pub fn has_returning(&self) -> bool { self.returning } + + pub fn table_version_id(&self) -> TableVersionId { + self.table_version_id + } } impl PlanTreeNodeUnary for LogicalUpdate { @@ -121,6 +136,7 @@ impl PlanTreeNodeUnary for LogicalUpdate { input, self.table_name.clone(), self.table_id, + self.table_version_id, self.exprs.clone(), self.returning, ) diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index afa2bd78607ed..c9baaf9f2770d 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -425,10 +425,11 @@ pub fn to_stream_prost_body( Distribution::Broadcast => DispatcherType::Broadcast, _ => panic!("Do not allow Any or AnyShard in serialization process"), } as i32, - column_indices: match &base.dist { + dist_key_indices: match &base.dist { Distribution::HashShard(keys) => keys.iter().map(|&num| num as u32).collect(), _ => vec![], }, + output_indices: (0..base.schema().len() as u32).collect(), }), }), Node::DynamicFilter(me) => { @@ -437,7 +438,7 @@ pub fn to_stream_prost_body( let condition = me .predicate() .as_expr_unless_true() - .map(|x| base.ctx().expr_with_session_timezone(x).to_expr_proto()); + .map(|x| x.to_expr_proto()); let left_table = infer_left_internal_table_catalog(base, me.left_index) .with_id(state.gen_table_id_wrapped()); let right_table = infer_right_internal_table_catalog(&me.right.0) @@ -483,7 +484,7 @@ pub fn to_stream_prost_body( .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| base.ctx().expr_with_session_timezone(x).to_expr_proto()), + .map(|x| x.to_expr_proto()), left_table_id: left_table_desc.table_id.table_id(), right_table_id: right_table_desc.table_id.table_id(), left_info: Some(ArrangementInfo { @@ -527,11 +528,7 @@ pub fn to_stream_prost_body( Node::Filter(me) => { let me = &me.core; ProstNode::Filter(FilterNode { - search_condition: Some( - base.ctx() - .expr_with_session_timezone(ExprImpl::from(me.predicate.clone())) - .to_expr_proto(), - ), + search_condition: Some(ExprImpl::from(me.predicate.clone()).to_expr_proto()), }) } Node::GlobalSimpleAgg(me) => { @@ -541,11 +538,7 @@ pub fn to_stream_prost_body( let distinct_dedup_tables = me.infer_distinct_dedup_tables(base, None); ProstNode::GlobalSimpleAgg(SimpleAggNode { - agg_calls: me - .agg_calls - .iter() - .map(|x| PlanAggCall::to_protobuf(x, base.ctx())) - .collect(), + agg_calls: me.agg_calls.iter().map(PlanAggCall::to_protobuf).collect(), distribution_key: base .dist .dist_column_indices() @@ -595,7 +588,7 @@ pub fn to_stream_prost_body( .core .agg_calls .iter() - .map(|x| PlanAggCall::to_protobuf(x, base.ctx())) + .map(PlanAggCall::to_protobuf) .collect(), is_append_only: me.core.input.0.append_only, @@ -661,7 +654,7 @@ pub fn to_stream_prost_body( .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| base.ctx().expr_with_session_timezone(x).to_expr_proto()), + .map(|x| x.to_expr_proto()), left_table: Some(left_table.to_internal_table_prost()), right_table: Some(right_table.to_internal_table_prost()), left_degree_table: Some(left_degree_table.to_internal_table_prost()), @@ -684,11 +677,7 @@ pub fn to_stream_prost_body( Node::LocalSimpleAgg(me) => { let me = &me.core; ProstNode::LocalSimpleAgg(SimpleAggNode { - agg_calls: me - .agg_calls - .iter() - .map(|x| PlanAggCall::to_protobuf(x, base.ctx())) - .collect(), + agg_calls: me.agg_calls.iter().map(PlanAggCall::to_protobuf).collect(), distribution_key: base .dist .dist_column_indices() @@ -721,16 +710,7 @@ pub fn to_stream_prost_body( ProstNode::ProjectSet(ProjectSetNode { select_list }) } Node::Project(me) => ProstNode::Project(ProjectNode { - select_list: me - .core - .exprs - .iter() - .map(|x| { - base.ctx() - .expr_with_session_timezone(x.clone()) - .to_expr_proto() - }) - .collect(), + select_list: me.core.exprs.iter().map(|x| x.to_expr_proto()).collect(), watermark_input_key: me .watermark_derivations .iter() diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs index 272e6115ddfa1..d0a6a63a25242 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -192,12 +192,7 @@ impl StreamNode for StreamDeltaJoin { .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) - .to_expr_proto() - }), + .map(|x| x.to_expr_proto()), left_table_id: left_table_desc.table_id.table_id(), right_table_id: right_table_desc.table_id.table_id(), left_info: Some(ArrangementInfo { diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs index 2c03e713145f4..4089b8d1e3ee1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dml.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dml.rs @@ -82,9 +82,8 @@ impl StreamNode for StreamDml { use risingwave_pb::stream_plan::*; ProstStreamNode::Dml(DmlNode { - // Meta will fill this table id. - table_id: 0, - table_version_id: INITIAL_TABLE_VERSION_ID, // TODO: use correct table version id + table_id: 0, // Meta will fill this table id. + table_version_id: INITIAL_TABLE_VERSION_ID, // Meta will fill this version id. column_descs: self.column_descs.iter().map(Into::into).collect(), }) } diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index e8f5d36cf53ed..a7619b75cc597 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -83,10 +83,11 @@ impl StreamNode for StreamExchange { Distribution::Broadcast => DispatcherType::Broadcast, _ => panic!("Do not allow Any or AnyShard in serialization process"), } as i32, - column_indices: match &self.base.dist { + dist_key_indices: match &self.base.dist { Distribution::HashShard(keys) => keys.iter().map(|num| *num as u32).collect(), _ => vec![], }, + output_indices: (0..self.schema().len() as u32).collect(), }), }) } diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 6599c69344576..c3c7aa77e3946 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -17,7 +17,6 @@ use std::fmt; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::FilterNode; -use super::generic::GenericPlanRef; use super::{ExprRewritable, LogicalFilter, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::PlanBase; @@ -76,12 +75,7 @@ impl_plan_tree_node_for_unary! { StreamFilter } impl StreamNode for StreamFilter { fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { ProstStreamNode::Filter(FilterNode { - search_condition: Some( - self.base - .ctx() - .expr_with_session_timezone(ExprImpl::from(self.predicate().clone())) - .to_expr_proto(), - ), + search_condition: Some(ExprImpl::from(self.predicate().clone()).to_expr_proto()), }) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs index 8846868b0ebc6..b659a946e75ea 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs @@ -20,7 +20,6 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::generic::PlanAggCall; use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -99,7 +98,7 @@ impl StreamNode for StreamGlobalSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(PlanAggCall::to_protobuf) .collect(), distribution_key: self .base diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index b8184bb627428..5adf9a8803386 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -20,7 +20,6 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::generic::PlanAggCall; use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -114,7 +113,7 @@ impl StreamNode for StreamHashAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(PlanAggCall::to_protobuf) .collect(), is_append_only: self.input().append_only(), diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index 3c6e9b72d80d8..f4a97229c93d6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -263,12 +263,7 @@ impl StreamNode for StreamHashJoin { .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) - .to_expr_proto() - }), + .map(|x| x.to_expr_proto()), left_table: Some(left_table.to_internal_table_prost()), right_table: Some(right_table.to_internal_table_prost()), left_degree_table: Some(left_degree_table.to_internal_table_prost()), diff --git a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs index 25fbe75d01733..42eb2da1c057b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs @@ -21,7 +21,6 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::generic::PlanAggCall; use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::property::RequiredDist; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -96,7 +95,7 @@ impl StreamNode for StreamLocalSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(PlanAggCall::to_protobuf) .collect(), distribution_key: self .distribution() diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 39f6f7edd1e9f..daad13346c5e3 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -235,6 +235,13 @@ impl fmt::Display for StreamMaterialize { if pk_column_names != order_descs { builder.field("order_descs", &format_args!("[{}]", order_descs)); } + + let pk_conflict_behavior = match self.table.handle_pk_conflict() { + true => "overwrite", + false => "no check", + }; + builder.field("pk_conflict", &pk_conflict_behavior); + builder.finish() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 8658db4889fde..6c1e67e43775b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -141,12 +141,7 @@ impl StreamNode for StreamProject { .logical .exprs() .iter() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x.clone()) - .to_expr_proto() - }) + .map(|x| x.to_expr_proto()) .collect(), watermark_input_key: self .watermark_derivations diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 66361b3b89174..3b6f24ff28796 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -85,33 +85,39 @@ impl StreamNode for StreamShare { impl StreamShare { pub fn adhoc_to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> ProstStreamPlan { let operator_id = self.base.id.0 as u32; + let output_indices = (0..self.schema().len() as u32).collect_vec(); + match state.get_share_stream_node(operator_id) { None => { let dispatch_strategy = match &self.base.dist { Distribution::HashShard(keys) | Distribution::UpstreamHashShard(keys, _) => { DispatchStrategy { r#type: DispatcherType::Hash as i32, - column_indices: keys.iter().map(|x| *x as u32).collect_vec(), + dist_key_indices: keys.iter().map(|x| *x as u32).collect_vec(), + output_indices, } } Distribution::Single => DispatchStrategy { r#type: DispatcherType::Simple as i32, - column_indices: vec![], + dist_key_indices: vec![], + output_indices, }, Distribution::Broadcast => DispatchStrategy { r#type: DispatcherType::Broadcast as i32, - column_indices: vec![], + dist_key_indices: vec![], + output_indices, }, Distribution::SomeShard => { // FIXME: use another DispatcherType? DispatchStrategy { r#type: DispatcherType::Hash as i32, - column_indices: self + dist_key_indices: self .base .logical_pk .iter() .map(|x| *x as u32) .collect_vec(), + output_indices, } } }; diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 0a5426aaa43fe..05881ffb73b8c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -28,6 +28,7 @@ use risingwave_connector::sink::{ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::derive::{derive_columns, derive_pk}; +use super::utils::IndicesDisplay; use super::{ExprRewritable, PlanBase, PlanRef, StreamNode}; use crate::optimizer::plan_node::PlanTreeNodeUnary; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -102,6 +103,14 @@ impl StreamSink { let sink_type = Self::derive_sink_type(input.append_only(), &properties)?; let (pk, stream_key) = derive_pk(input, user_order_by, &columns); + if sink_type == SinkType::Upsert && pk.is_empty() { + return Err(ErrorCode::SinkError(Box::new(Error::new( + ErrorKind::InvalidInput, + "No primary key for the upsert sink. Please include the primary key explicitly in sink definition or make the sink append-only.", + ))) + .into()); + } + Ok(SinkDesc { id: SinkId::placeholder(), name, @@ -140,7 +149,7 @@ impl StreamSink { (_, false, true) => { Err(ErrorCode::SinkError(Box::new(Error::new( ErrorKind::InvalidInput, - "Cannot force the sink to be append-only without \"format='append_only'\"in WITH options", + "Cannot force the sink to be append-only without \"format='append_only'\"in WITH options.", ))) .into()) } @@ -164,6 +173,33 @@ impl_plan_tree_node_for_unary! { StreamSink } impl fmt::Display for StreamSink { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let mut builder = f.debug_struct("StreamSink"); + + let sink_type = if self.sink_desc.sink_type.is_append_only() { + "append-only" + } else { + "upsert" + }; + let column_names = self + .sink_desc + .columns + .iter() + .map(|col| col.column_desc.name.clone()) + .collect_vec() + .join(", "); + builder + .field("type", &format_args!("{}", sink_type)) + .field("columns", &format_args!("[{}]", column_names)); + + if self.sink_desc.sink_type.is_upsert() { + builder.field( + "pk", + &IndicesDisplay { + indices: &self.sink_desc.pk.iter().map(|k| k.column_idx).collect_vec(), + input_schema: &self.base.schema, + }, + ); + } + builder.finish() } } 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 d53d51fa110b3..e89079d03e224 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -246,12 +246,13 @@ impl ExprRewritable for StreamTableScan { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self::new( + Self::new_with_chain_type( self.logical .rewrite_exprs(r) .as_logical_scan() .unwrap() .clone(), + self.chain_type, ) .into() } diff --git a/src/frontend/src/planner/delete.rs b/src/frontend/src/planner/delete.rs index 9814d364e628a..68cb349281b01 100644 --- a/src/frontend/src/planner/delete.rs +++ b/src/frontend/src/planner/delete.rs @@ -30,9 +30,14 @@ impl Planner { scan }; let returning = !delete.returning_list.is_empty(); - let mut plan: PlanRef = - LogicalDelete::create(input, delete.table_name.clone(), delete.table_id, returning)? - .into(); + let mut plan: PlanRef = LogicalDelete::create( + input, + delete.table_name.clone(), + delete.table_id, + delete.table_version_id, + returning, + )? + .into(); if returning { plan = LogicalProject::create(plan, delete.returning_list); diff --git a/src/frontend/src/planner/insert.rs b/src/frontend/src/planner/insert.rs index c7b94d34195e8..11501764b504f 100644 --- a/src/frontend/src/planner/insert.rs +++ b/src/frontend/src/planner/insert.rs @@ -32,6 +32,7 @@ impl Planner { input, insert.table_name.clone(), insert.table_id, + insert.table_version_id, insert.column_indices, insert.row_id_index, returning, diff --git a/src/frontend/src/planner/update.rs b/src/frontend/src/planner/update.rs index 2dfca750aee99..8b79d00d4a834 100644 --- a/src/frontend/src/planner/update.rs +++ b/src/frontend/src/planner/update.rs @@ -35,6 +35,7 @@ impl Planner { input, update.table_name.clone(), update.table_id, + update.table_version_id, update.exprs, returning, )? diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 1e106e2e8806b..bf17453cafb68 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -16,7 +16,6 @@ use std::collections::HashMap; use std::io::{Error, ErrorKind}; use std::sync::atomic::{AtomicI32, Ordering}; use std::sync::{Arc, Mutex}; -// use tokio::sync::Mutex; use std::time::Duration; use parking_lot::{RwLock, RwLockReadGuard}; diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 35eae9dc34d77..628ba0d486f25 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -139,7 +139,8 @@ fn rewrite_stream_node( let strategy = DispatchStrategy { r#type: DispatcherType::NoShuffle.into(), - column_indices: vec![], // TODO: use distribution key + dist_key_indices: vec![], // TODO: use distribution key + output_indices: (0..(child_node.fields.len() as u32)).collect(), }; Ok(StreamNode { stream_key: child_node.stream_key.clone(), diff --git a/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs b/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs index fec3424c2ac00..1450b373baa24 100644 --- a/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs +++ b/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs @@ -36,7 +36,9 @@ fn build_no_shuffle_exchange_for_delta_join( fields: upstream.fields.clone(), stream_key: upstream.stream_key.clone(), node_body: Some(NodeBody::Exchange(ExchangeNode { - strategy: Some(dispatch_no_shuffle()), + strategy: Some(dispatch_no_shuffle( + (0..(upstream.fields.len() as u32)).collect(), + )), })), input: vec![], append_only: upstream.append_only, @@ -46,7 +48,7 @@ fn build_no_shuffle_exchange_for_delta_join( fn build_consistent_hash_shuffle_exchange_for_delta_join( state: &mut BuildFragmentGraphState, upstream: &StreamNode, - column_indices: Vec, + dist_key_indices: Vec, ) -> StreamNode { StreamNode { operator_id: state.gen_operator_id() as u64, @@ -54,25 +56,33 @@ fn build_consistent_hash_shuffle_exchange_for_delta_join( fields: upstream.fields.clone(), stream_key: upstream.stream_key.clone(), node_body: Some(NodeBody::Exchange(ExchangeNode { - strategy: Some(dispatch_consistent_hash_shuffle(column_indices)), + strategy: Some(dispatch_consistent_hash_shuffle( + dist_key_indices, + (0..(upstream.fields.len() as u32)).collect(), + )), })), input: vec![], append_only: upstream.append_only, } } -fn dispatch_no_shuffle() -> DispatchStrategy { +fn dispatch_no_shuffle(output_indices: Vec) -> DispatchStrategy { DispatchStrategy { r#type: DispatcherType::NoShuffle.into(), - column_indices: vec![], + dist_key_indices: vec![], + output_indices, } } -fn dispatch_consistent_hash_shuffle(column_indices: Vec) -> DispatchStrategy { +fn dispatch_consistent_hash_shuffle( + dist_key_indices: Vec, + output_indices: Vec, +) -> DispatchStrategy { // Actually Hash shuffle is consistent hash shuffle now. DispatchStrategy { r#type: DispatcherType::Hash.into(), - column_indices, + dist_key_indices, + output_indices, } } @@ -136,6 +146,9 @@ fn build_delta_join_inner( let i0_length = arrange_0.fields.len(); let i1_length = arrange_1.fields.len(); + let i0_output_indices = (0..i0_length as u32).collect_vec(); + let i1_output_indices = (0..i1_length as u32).collect_vec(); + let lookup_0_column_reordering = { let tmp: Vec = (i1_length..i1_length + i0_length) .chain(0..i1_length) @@ -204,7 +217,7 @@ fn build_delta_join_inner( arrange_0_frag.fragment_id, lookup_0_frag.fragment_id, StreamFragmentEdge { - dispatch_strategy: dispatch_no_shuffle(), + dispatch_strategy: dispatch_no_shuffle(i0_output_indices.clone()), link_id: exchange_a0l0.operator_id, }, ); @@ -221,6 +234,7 @@ fn build_delta_join_inner( .iter() .map(|x| *x as u32) .collect_vec(), + i0_output_indices, ), link_id: exchange_a0l1.operator_id, }, @@ -238,6 +252,7 @@ fn build_delta_join_inner( .iter() .map(|x| *x as u32) .collect_vec(), + i1_output_indices.clone(), ), link_id: exchange_a1l0.operator_id, }, @@ -249,7 +264,7 @@ fn build_delta_join_inner( arrange_1_frag.fragment_id, lookup_1_frag.fragment_id, StreamFragmentEdge { - dispatch_strategy: dispatch_no_shuffle(), + dispatch_strategy: dispatch_no_shuffle(i1_output_indices), link_id: exchange_a1l1.operator_id, }, ); @@ -275,7 +290,10 @@ fn build_delta_join_inner( lookup_0_frag.fragment_id, current_fragment.fragment_id, StreamFragmentEdge { - dispatch_strategy: dispatch_consistent_hash_shuffle(node.stream_key.clone()), + dispatch_strategy: dispatch_consistent_hash_shuffle( + node.stream_key.clone(), + (0..node.fields.len() as u32).collect(), + ), link_id: exchange_l0m.operator_id, }, ); @@ -284,7 +302,10 @@ fn build_delta_join_inner( lookup_1_frag.fragment_id, current_fragment.fragment_id, StreamFragmentEdge { - dispatch_strategy: dispatch_consistent_hash_shuffle(node.stream_key.clone()), + dispatch_strategy: dispatch_consistent_hash_shuffle( + node.stream_key.clone(), + (0..node.fields.len() as u32).collect(), + ), link_id: exchange_l1m.operator_id, }, ); diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index db227c01b7520..8aa46fb9c8209 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -27,6 +27,7 @@ use risingwave_common::catalog::{ DEFAULT_SUPER_USER_ID, NON_RESERVED_USER_ID, PG_CATALOG_SCHEMA_NAME, }; use risingwave_common::error::Result; +use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_pb::backup_service::MetaSnapshotMetadata; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ @@ -42,7 +43,6 @@ use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_pb::user::update_user_request::UpdateField; use risingwave_pb::user::{GrantPrivilege, UserInfo}; use risingwave_rpc_client::error::Result as RpcResult; -use risingwave_rpc_client::SystemParamsReader; use tempfile::{Builder, NamedTempFile}; use crate::catalog::catalog_service::CatalogWriter; diff --git a/src/frontend/src/utils/column_index_mapping.rs b/src/frontend/src/utils/column_index_mapping.rs index 73c19a9b754e3..4757ace97fd40 100644 --- a/src/frontend/src/utils/column_index_mapping.rs +++ b/src/frontend/src/utils/column_index_mapping.rs @@ -1,4 +1,3 @@ -use std::cmp::max; // Copyright 2023 RisingWave Labs // // Licensed under the Apache License, Version 2.0 (the "License"); @@ -12,6 +11,7 @@ use std::cmp::max; // 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::cmp::max; use std::fmt::Debug; use std::vec; diff --git a/src/frontend/src/utils/stream_graph_formatter.rs b/src/frontend/src/utils/stream_graph_formatter.rs index 017c76593a740..54c8140ae4d8e 100644 --- a/src/frontend/src/utils/stream_graph_formatter.rs +++ b/src/frontend/src/utils/stream_graph_formatter.rs @@ -129,7 +129,7 @@ impl StreamGraphFormatter { "StreamExchange {} from {}", match dist.r#type() { DispatcherType::Unspecified => unreachable!(), - DispatcherType::Hash => format!("Hash({:?})", dist.column_indices), + DispatcherType::Hash => format!("Hash({:?})", dist.dist_key_indices), DispatcherType::Broadcast => "Broadcast".to_string(), DispatcherType::Simple => "Single".to_string(), DispatcherType::NoShuffle => "NoShuffle".to_string(), diff --git a/src/java_binding/make-java-binding.toml b/src/java_binding/make-java-binding.toml index 3240d4a59301a..f2dc910978201 100644 --- a/src/java_binding/make-java-binding.toml +++ b/src/java_binding/make-java-binding.toml @@ -24,16 +24,55 @@ cd src/java_binding/java mvn clean install ''' +[tasks.start-java-binding-demo-cluster] +description = "Start the RisingWave cluster for java binding demo" +script = ''' +#!/usr/bin/env bash +set -ex + +RISINGWAVE_ROOT=$(git rev-parse --show-toplevel) + +echo 'start risingwave cluster' +${RISINGWAVE_ROOT}/risedev d java-binding-demo +''' + +[tasks.kill-java-binding-demo-cluster] +description = "Kill the RisingWave cluster for java binding demo" +script = ''' +#!/usr/bin/env bash +set -ex + +export RISINGWAVE_ROOT=$(git rev-parse --show-toplevel) + +echo 'kill risingwave cluster' +${RISINGWAVE_ROOT}/risedev k > /dev/null +''' + +[tasks.ingest-data-and-run-java-binding] +description = "Ingest test data and run java binding demo" +script = ''' +#!/usr/bin/env bash +set -ex + +export RISINGWAVE_ROOT=$(git rev-parse --show-toplevel) +export JAVA_BINDING_ROOT=${RISINGWAVE_ROOT}/src/java_binding +export TABLE_NAME=java_binding_demo +export DB_NAME=dev +# Below variables are determined by risedev. +# See the `java-binding-demo` section in risedev.yml. +export OBJECT_STORE=minio://hummockadmin:hummockadmin@127.0.0.1:9301/hummock001 +export META_ADDR=127.0.0.1:5690 +export DATA_DIR=hummock_001 + +bash ${JAVA_BINDING_ROOT}/run_demo.sh +''' + [tasks.run-java-binding-demo] description = "Run the java binding demo" dependencies = [ "build-java-binding-rust", - "build-java-binding-java" + "build-java-binding-java", + "start-java-binding-demo-cluster", + "ingest-data-and-run-java-binding", + "kill-java-binding-demo-cluster" ] -script = ''' -#!/usr/bin/env bash -set -ex -RISINGWAVE_ROOT=$(git rev-parse --show-toplevel) -JAVA_BINDING_ROOT=${RISINGWAVE_ROOT}/src/java_binding -source ${JAVA_BINDING_ROOT}/run_demo.sh -''' \ No newline at end of file diff --git a/src/java_binding/run_demo.sh b/src/java_binding/run_demo.sh index 51cd4bc70a036..80ed20fc2dfee 100644 --- a/src/java_binding/run_demo.sh +++ b/src/java_binding/run_demo.sh @@ -2,17 +2,7 @@ # This script must be executed from `cargo make run-java-binding-demo`. -set -e - -TABLE_NAME=java_binding_demo -DB_NAME=dev -# Below variables are determined by risedev. -# See the `java-binding-demo` section in risedev.yml. -OBJECT_STORE=minio://hummockadmin:hummockadmin@127.0.0.1:9301/hummock001 -META_ADDR=127.0.0.1:5690 -DATA_DIR=hummock_001 - -${RISINGWAVE_ROOT}/risedev d java-binding-demo +set -ex psql -d ${DB_NAME} -h localhost -p 4566 -U root << EOF DROP TABLE IF EXISTS ${TABLE_NAME}; @@ -23,11 +13,6 @@ EOF cd ${JAVA_BINDING_ROOT}/java -TABLE_NAME=${TABLE_NAME} \ -DB_NAME=${DB_NAME} \ -OBJECT_STORE=${OBJECT_STORE} \ -META_ADDR=${META_ADDR} \ -DATA_DIR=${DATA_DIR} \ mvn exec:exec \ -pl java-binding \ -Dexec.executable=java \ @@ -37,7 +22,4 @@ mvn exec:exec \ psql -d dev -h localhost -p 4566 -U root << EOF DROP TABLE ${TABLE_NAME}; -EOF - -cd - -${RISINGWAVE_ROOT}/risedev k > /dev/null +EOF \ No newline at end of file diff --git a/src/meta/src/backup_restore/restore.rs b/src/meta/src/backup_restore/restore.rs index cf01331c34b48..c52fff6a4532e 100644 --- a/src/meta/src/backup_restore/restore.rs +++ b/src/meta/src/backup_restore/restore.rs @@ -198,8 +198,8 @@ async fn restore_impl( tracing::info!( "snapshot {} after rewrite by snapshot {}:\n{}", target_id, + newest_id, target_snapshot, - newest_id ); } if opts.dry_run { diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index b570fb91f8186..3cd348debd247 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -52,7 +52,8 @@ use crate::barrier::snapshot::SnapshotManager; use crate::barrier::BarrierEpochState::{Completed, InFlight}; use crate::hummock::HummockManagerRef; use crate::manager::{ - CatalogManagerRef, ClusterManagerRef, FragmentManagerRef, MetaSrvEnv, WorkerId, + CatalogManagerRef, ClusterManagerRef, FragmentManagerRef, LocalNotification, MetaSrvEnv, + WorkerId, }; use crate::model::{ActorId, BarrierManagerState}; use crate::rpc::metrics::MetaMetrics; @@ -564,6 +565,12 @@ where let mut barrier_timer: Option = None; let (barrier_complete_tx, mut barrier_complete_rx) = tokio::sync::mpsc::unbounded_channel(); let mut checkpoint_control = CheckpointControl::new(self.metrics.clone()); + let (local_notification_tx, mut local_notification_rx) = + tokio::sync::mpsc::unbounded_channel(); + self.env + .notification_manager() + .insert_local_sender(local_notification_tx) + .await; loop { tokio::select! { biased; @@ -572,6 +579,10 @@ where tracing::info!("Barrier manager is stopped"); return; } + // Checkpoint frequency change + notification = local_notification_rx.recv() => { + self.handle_local_notification(notification.unwrap()); + } result = barrier_complete_rx.recv() => { checkpoint_control.update_barrier_nums_metrics(); @@ -982,6 +993,14 @@ where pub async fn get_ddl_progress(&self) -> Vec { self.tracker.lock().await.gen_ddl_progress() } + + /// Only handle `SystemParamsChange`. + fn handle_local_notification(&self, notification: LocalNotification) { + if let LocalNotification::SystemParamsChange(p) = notification { + self.scheduled_barriers + .set_checkpoint_frequency(p.checkpoint_frequency() as usize) + } + } } pub type BarrierManagerRef = Arc>; diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index 5dc62ed57618a..ce5536251d740 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -45,7 +45,7 @@ struct Inner { /// Force checkpoint in next barrier. force_checkpoint: AtomicBool, - checkpoint_frequency: usize, + checkpoint_frequency: AtomicUsize, } /// The sender side of the barrier scheduling queue. @@ -73,7 +73,7 @@ impl BarrierScheduler { queue: RwLock::new(VecDeque::new()), changed_tx: watch::channel(()).0, num_uncheckpointed_barrier: AtomicUsize::new(0), - checkpoint_frequency, + checkpoint_frequency: AtomicUsize::new(checkpoint_frequency), force_checkpoint: AtomicBool::new(false), }); @@ -281,7 +281,7 @@ impl ScheduledBarriers { self.inner .num_uncheckpointed_barrier .load(Ordering::Relaxed) - >= self.inner.checkpoint_frequency + >= self.inner.checkpoint_frequency.load(Ordering::Relaxed) || self.inner.force_checkpoint.load(Ordering::Relaxed) } @@ -290,6 +290,13 @@ impl ScheduledBarriers { self.inner.force_checkpoint.store(true, Ordering::Relaxed) } + /// Update the `checkpoint_frequency` + pub fn set_checkpoint_frequency(&self, frequency: usize) { + self.inner + .checkpoint_frequency + .store(frequency, Ordering::Relaxed); + } + /// Update the `num_uncheckpointed_barrier` fn update_num_uncheckpointed_barrier(&self, checkpoint: bool) { if checkpoint { diff --git a/src/meta/src/hummock/compaction/level_selector.rs b/src/meta/src/hummock/compaction/level_selector.rs index 71885539f015c..2f423dacd7be0 100644 --- a/src/meta/src/hummock/compaction/level_selector.rs +++ b/src/meta/src/hummock/compaction/level_selector.rs @@ -19,6 +19,7 @@ use std::collections::HashMap; // (found in the LICENSE.Apache file in the root directory). use std::sync::Arc; +use risingwave_common::catalog::TableOption; use risingwave_hummock_sdk::HummockCompactionTaskId; use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::{compact_task, CompactionConfig}; @@ -50,6 +51,7 @@ pub trait LevelSelector: Sync + Send { levels: &Levels, level_handlers: &mut [LevelHandler], selector_stats: &mut LocalSelectorStatistic, + table_id_to_options: HashMap, ) -> Option; fn report_statistic_metrics(&self, _metrics: &MetaMetrics) {} @@ -246,6 +248,7 @@ impl LevelSelector for DynamicLevelSelector { levels: &Levels, level_handlers: &mut [LevelHandler], selector_stats: &mut LocalSelectorStatistic, + _table_id_to_options: HashMap, ) -> Option { let dynamic_level_core = DynamicLevelSelectorCore::new(compaction_group.compaction_config.clone()); @@ -305,6 +308,7 @@ impl LevelSelector for ManualCompactionSelector { levels: &Levels, level_handlers: &mut [LevelHandler], _selector_stats: &mut LocalSelectorStatistic, + _table_id_to_options: HashMap, ) -> Option { let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); let overlap_strategy = create_overlap_strategy(group.compaction_config.compaction_mode()); @@ -360,6 +364,7 @@ impl LevelSelector for SpaceReclaimCompactionSelector { levels: &Levels, level_handlers: &mut [LevelHandler], _selector_stats: &mut LocalSelectorStatistic, + _table_id_to_options: HashMap, ) -> Option { let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); let mut picker = SpaceReclaimCompactionPicker::new( @@ -404,11 +409,14 @@ impl LevelSelector for TtlCompactionSelector { levels: &Levels, level_handlers: &mut [LevelHandler], _selector_stats: &mut LocalSelectorStatistic, + table_id_to_options: HashMap, ) -> Option { let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); let ctx = dynamic_level_core.calculate_level_base_size(levels); - let picker = - TtlReclaimCompactionPicker::new(group.compaction_config.max_space_reclaim_bytes); + let picker = TtlReclaimCompactionPicker::new( + group.compaction_config.max_space_reclaim_bytes, + table_id_to_options, + ); let state = self .state .entry(group.group_id) @@ -506,16 +514,21 @@ pub mod tests { stale_key_count: 0, total_key_count: 0, divide_version: 0, + min_epoch: 0, + max_epoch: 0, } } - pub fn generate_table_with_table_ids( + #[allow(clippy::too_many_arguments)] + pub fn generate_table_with_ids_and_epochs( id: u64, table_prefix: u64, left: usize, right: usize, epoch: u64, table_ids: Vec, + min_epoch: u64, + max_epoch: u64, ) -> SstableInfo { SstableInfo { id, @@ -530,6 +543,8 @@ pub mod tests { stale_key_count: 0, total_key_count: 0, divide_version: 0, + min_epoch, + max_epoch, } } @@ -726,6 +741,7 @@ pub mod tests { &levels, &mut levels_handlers, &mut local_stats, + HashMap::default(), ) .unwrap(); // trivial move. @@ -753,6 +769,7 @@ pub mod tests { &levels, &mut levels_handlers, &mut local_stats, + HashMap::default(), ) .unwrap(); assert_compaction_task(&compaction, &levels_handlers); @@ -771,6 +788,7 @@ pub mod tests { &levels, &mut levels_handlers, &mut local_stats, + HashMap::default(), ) .unwrap(); assert_compaction_task(&compaction, &levels_handlers); @@ -791,6 +809,7 @@ pub mod tests { &levels, &mut levels_handlers, &mut local_stats, + HashMap::default(), ); assert!(compaction.is_none()); } diff --git a/src/meta/src/hummock/compaction/mod.rs b/src/meta/src/hummock/compaction/mod.rs index 2c015e44760a7..4b90d05276060 100644 --- a/src/meta/src/hummock/compaction/mod.rs +++ b/src/meta/src/hummock/compaction/mod.rs @@ -15,6 +15,7 @@ pub mod compaction_config; mod level_selector; mod overlap_strategy; +use risingwave_common::catalog::TableOption; use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; use risingwave_pb::hummock::compact_task::{self, TaskStatus}; @@ -121,12 +122,19 @@ impl CompactStatus { group: &CompactionGroup, stats: &mut LocalSelectorStatistic, selector: &mut Box, + table_id_to_options: HashMap, ) -> Option { // When we compact the files, we must make the result of compaction meet the following // conditions, for any user key, the epoch of it in the file existing in the lower // layer must be larger. - let ret = - selector.pick_compaction(task_id, group, levels, &mut self.level_handlers, stats)?; + let ret = selector.pick_compaction( + task_id, + group, + levels, + &mut self.level_handlers, + stats, + table_id_to_options, + )?; let target_level_id = ret.input.target_level; let compression_algorithm = match ret.compression_algorithm.as_str() { diff --git a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs index a511dc30f6be6..60b8c175c7c5b 100644 --- a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs @@ -319,7 +319,7 @@ impl CompactionPicker for ManualCompactionPicker { #[cfg(test)] pub mod tests { - use std::collections::HashSet; + use std::collections::{HashMap, HashSet}; use risingwave_pb::hummock::compact_task; pub use risingwave_pb::hummock::{KeyRange, Level, LevelType}; @@ -1196,6 +1196,7 @@ pub mod tests { &levels, &mut levels_handler, &mut local_stats, + HashMap::default(), ) .unwrap(); assert_compaction_task(&task, &levels_handler); @@ -1231,6 +1232,7 @@ pub mod tests { &levels, &mut levels_handler, &mut local_stats, + HashMap::default(), ) .unwrap(); assert_compaction_task(&task, &levels_handler); @@ -1303,6 +1305,7 @@ pub mod tests { &levels, &mut levels_handler, &mut local_stats, + HashMap::default(), ) .unwrap(); assert_compaction_task(&task, &levels_handler); @@ -1340,6 +1343,7 @@ pub mod tests { &levels, &mut levels_handler, &mut local_stats, + HashMap::default(), ) .unwrap(); assert_compaction_task(&task, &levels_handler); diff --git a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs index bdb4937ce4da4..a193a05065d00 100644 --- a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs @@ -14,21 +14,56 @@ use std::collections::HashSet; +use risingwave_hummock_sdk::key_range::KeyRangeCommon; use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{InputLevel, SstableInfo}; +use risingwave_pb::hummock::{InputLevel, KeyRange, SstableInfo}; use crate::hummock::compaction::CompactionInput; use crate::hummock::level_handler::LevelHandler; +// The execution model of SpaceReclaimCompactionPicker scans through the last level of files by +// key_range and selects the appropriate files to generate compaction pub struct SpaceReclaimCompactionPicker { // config pub max_space_reclaim_bytes: u64, + + // for filter pub all_table_ids: HashSet, } +// According to the execution model of SpaceReclaimCompactionPicker, SpaceReclaimPickerState is +// designed to record the state of each round of scanning #[derive(Default)] pub struct SpaceReclaimPickerState { - pub last_select_index: usize, + // Because of the right_exclusive, we use KeyRangeCommon to determine if the end_bounds + // overlap instead of directly comparing Vec. We don't need to use the start_bound in the + // filter, set it to -inf + + // record the end_bound that has been scanned + pub last_select_end_bound: KeyRange, + + // record the end_bound in the current round of scanning tasks + pub end_bound_in_round: KeyRange, +} + +impl SpaceReclaimPickerState { + pub fn valid(&self) -> bool { + !self.end_bound_in_round.right.is_empty() + } + + pub fn init(&mut self, key_range: KeyRange) { + self.last_select_end_bound = KeyRange { + left: vec![], + right: key_range.left.clone(), + right_exclusive: true, + }; + self.end_bound_in_round = key_range; + } + + pub fn clear(&mut self) { + self.end_bound_in_round = KeyRange::default(); + self.last_select_end_bound = KeyRange::default(); + } } impl SpaceReclaimCompactionPicker { @@ -60,16 +95,52 @@ impl SpaceReclaimCompactionPicker { let mut select_input_ssts = vec![]; let level_handler = &level_handlers[reclaimed_level.level_idx as usize]; - if state.last_select_index >= reclaimed_level.table_infos.len() { - state.last_select_index = 0; + if reclaimed_level.table_infos.is_empty() { + // no file to be picked + state.clear(); + return None; + } + + if state.valid() + && state + .last_select_end_bound + .compare_right_with(&state.end_bound_in_round.right) + == std::cmp::Ordering::Greater + { + // in round but end_key overflow + // turn to next_round + state.clear(); + return None; + } + + if !state.valid() { + // new round init key_range bound with table_infos + let first_sst = reclaimed_level.table_infos.first().unwrap(); + let last_sst = reclaimed_level.table_infos.last().unwrap(); + + let key_range_this_round = KeyRange { + left: first_sst.key_range.as_ref().unwrap().left.clone(), + right: last_sst.key_range.as_ref().unwrap().right.clone(), + right_exclusive: last_sst.key_range.as_ref().unwrap().right_exclusive, + }; + state.init(key_range_this_round); } - let start_indedx = state.last_select_index; let mut select_file_size = 0; + for sst in &reclaimed_level.table_infos { + let unmatched_sst = sst + .key_range + .as_ref() + .unwrap() + .sstable_overlap(&state.last_select_end_bound); + if unmatched_sst || (level_handler.is_pending_compact(&sst.id) || self.filter(sst)) { + if !select_input_ssts.is_empty() { + // Our goal is to pick as many complete layers of data as possible and keep the + // picked files contiguous to avoid overlapping key_ranges, so the strategy is + // to pick as many contiguous files as possible (at least one) + break; + } - for sst in &reclaimed_level.table_infos[start_indedx..] { - state.last_select_index += 1; - if level_handler.is_pending_compact(&sst.id) || self.filter(sst) { continue; } @@ -80,10 +151,19 @@ impl SpaceReclaimCompactionPicker { } } + // turn to next_round if select_input_ssts.is_empty() { + state.clear(); return None; } + let select_last_sst = select_input_ssts.last().unwrap(); + state.last_select_end_bound.full_key_extend(&KeyRange { + left: vec![], + right: select_last_sst.key_range.as_ref().unwrap().right.clone(), + right_exclusive: select_last_sst.key_range.as_ref().unwrap().right_exclusive, + }); + Some(CompactionInput { input_levels: vec![ InputLevel { @@ -106,6 +186,8 @@ impl SpaceReclaimCompactionPicker { #[cfg(test)] mod test { + use std::collections::HashMap; + use itertools::Itertools; use risingwave_pb::hummock::compact_task; pub use risingwave_pb::hummock::{KeyRange, Level, LevelType}; @@ -114,7 +196,7 @@ mod test { use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; use crate::hummock::compaction::level_selector::tests::{ assert_compaction_task, generate_l0_nonoverlapping_sublevels, generate_level, - generate_table_with_table_ids, + generate_table_with_ids_and_epochs, }; use crate::hummock::compaction::level_selector::SpaceReclaimCompactionSelector; use crate::hummock::compaction::{LevelSelector, LocalSelectorStatistic}; @@ -131,34 +213,42 @@ mod test { let l0 = generate_l0_nonoverlapping_sublevels(vec![]); assert_eq!(l0.sub_levels.len(), 0); - let levels = vec![ + let mut levels = vec![ generate_level(1, vec![]), generate_level(2, vec![]), generate_level( 3, vec![ - generate_table_with_table_ids(0, 1, 150, 151, 1, vec![0]), - generate_table_with_table_ids(1, 1, 250, 251, 1, vec![1]), + generate_table_with_ids_and_epochs(0, 1, 150, 151, 1, vec![0], 0, 0), + generate_table_with_ids_and_epochs(1, 1, 250, 251, 1, vec![1], 0, 0), ], ), Level { level_idx: 4, level_type: LevelType::Nonoverlapping as i32, table_infos: vec![ - generate_table_with_table_ids(2, 1, 0, 100, 1, vec![2]), - generate_table_with_table_ids(3, 1, 101, 200, 1, vec![3]), - generate_table_with_table_ids(4, 1, 222, 300, 1, vec![4]), - generate_table_with_table_ids(5, 1, 333, 400, 1, vec![5]), - generate_table_with_table_ids(6, 1, 444, 500, 1, vec![6]), - generate_table_with_table_ids(7, 1, 555, 600, 1, vec![7]), - generate_table_with_table_ids(8, 1, 666, 700, 1, vec![8]), - generate_table_with_table_ids(9, 1, 777, 800, 1, vec![9]), - generate_table_with_table_ids(10, 1, 888, 900, 1, vec![10]), + generate_table_with_ids_and_epochs(2, 1, 0, 100, 1, vec![2], 0, 0), + generate_table_with_ids_and_epochs(3, 1, 101, 200, 1, vec![3], 0, 0), + generate_table_with_ids_and_epochs(4, 1, 222, 300, 1, vec![4], 0, 0), + generate_table_with_ids_and_epochs(5, 1, 333, 400, 1, vec![5], 0, 0), + generate_table_with_ids_and_epochs(6, 1, 444, 500, 1, vec![6], 0, 0), + generate_table_with_ids_and_epochs(7, 1, 555, 600, 1, vec![7], 0, 0), + generate_table_with_ids_and_epochs(8, 1, 666, 700, 1, vec![8], 0, 0), + generate_table_with_ids_and_epochs(9, 1, 777, 800, 1, vec![9], 0, 0), + generate_table_with_ids_and_epochs(10, 1, 888, 1600, 1, vec![10], 0, 0), + generate_table_with_ids_and_epochs(11, 1, 1600, 1800, 1, vec![10], 0, 0), ], total_file_size: 0, sub_level_id: 0, }, ]; + + { + let sst_10 = levels[3].table_infos.get_mut(8).unwrap(); + assert_eq!(10, sst_10.id); + sst_10.key_range.as_mut().unwrap().right_exclusive = true; + } + assert_eq!(levels.len(), 4); let mut levels = Levels { levels, @@ -170,6 +260,8 @@ mod test { let mut selector = SpaceReclaimCompactionSelector::default(); { + // test max_pick_files limit + // pick space reclaim let task = selector .pick_compaction( @@ -178,6 +270,7 @@ mod test { &levels, &mut levels_handler, &mut local_stats, + HashMap::default(), ) .unwrap(); assert_compaction_task(&task, &levels_handler); @@ -209,6 +302,7 @@ mod test { } { + // test state for level_handler in &mut levels_handler { for pending_task_id in &level_handler.pending_tasks_ids() { level_handler.remove_task(*pending_task_id); @@ -223,18 +317,15 @@ mod test { &levels, &mut levels_handler, &mut local_stats, + HashMap::default(), ) .unwrap(); assert_compaction_task(&task, &levels_handler); assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 4); - // test select index, picker will select file from last_select_index - let all_file_count = levels.get_levels().last().unwrap().get_table_infos().len(); - assert_eq!( - task.input.input_levels[0].table_infos.len(), - all_file_count - 5 - ); + // test select index, picker will select file from state + assert_eq!(task.input.input_levels[0].table_infos.len(), 4,); let mut start_id = 7; for sst in &task.input.input_levels[0].table_infos { @@ -249,9 +340,49 @@ mod test { task.compaction_task_type, compact_task::TaskType::SpaceReclaim )); + + // test pick key_range right exclusive + let task = selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handler, + &mut local_stats, + HashMap::default(), + ) + .unwrap(); + assert_compaction_task(&task, &levels_handler); + assert_eq!(task.input.input_levels.len(), 2); + assert_eq!(task.input.input_levels[0].level_idx, 4); + assert_eq!(task.input.input_levels[0].table_infos.len(), 1); + assert_eq!(task.input.input_levels[1].level_idx, 4); + assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.target_level, 4); + assert!(matches!( + task.compaction_task_type, + compact_task::TaskType::SpaceReclaim + )); + for sst in &task.input.input_levels[0].table_infos { + assert_eq!(start_id, sst.id); + start_id += 1; + } + + assert!(selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handler, + &mut local_stats, + HashMap::default(), + ) + .is_none()) } { + // test state, after above 2 + for level_handler in &mut levels_handler { for pending_task_id in &level_handler.pending_tasks_ids() { level_handler.remove_task(*pending_task_id); @@ -266,6 +397,7 @@ mod test { &levels, &mut levels_handler, &mut local_stats, + HashMap::default(), ); assert!(task.is_none()); } @@ -286,6 +418,7 @@ mod test { &levels, &mut levels_handler, &mut local_stats, + HashMap::default(), ) .unwrap(); assert_compaction_task(&task, &levels_handler); @@ -308,5 +441,109 @@ mod test { compact_task::TaskType::SpaceReclaim )); } + + { + // test continuous file selection + for level_handler in &mut levels_handler { + for pending_task_id in &level_handler.pending_tasks_ids() { + level_handler.remove_task(*pending_task_id); + } + } + + // rebuild selector + selector = SpaceReclaimCompactionSelector::default(); + // cut range [3,4] [6] [8,9,10] + levels.member_table_ids = vec![2, 5, 7]; + let expect_task_file_count = vec![2, 1, 3]; + let expect_task_sst_id_range = vec![vec![3, 4], vec![6], vec![8, 9, 10]]; + for (index, x) in expect_task_file_count.iter().enumerate() { + // // pick space reclaim + let task = selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handler, + &mut local_stats, + HashMap::default(), + ) + .unwrap(); + + assert_compaction_task(&task, &levels_handler); + assert_eq!(task.input.input_levels.len(), 2); + assert_eq!(task.input.input_levels[0].level_idx, 4); + + assert_eq!(task.input.input_levels[0].table_infos.len(), *x); + let select_sst = &task.input.input_levels[0] + .table_infos + .iter() + .map(|sst| sst.id) + .collect_vec(); + assert!(select_sst.is_sorted()); + assert_eq!(expect_task_sst_id_range[index], *select_sst); + + assert_eq!(task.input.input_levels[1].level_idx, 4); + assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.target_level, 4); + assert!(matches!( + task.compaction_task_type, + compact_task::TaskType::SpaceReclaim + )); + } + } + + { + // test continuous file selection with filter change + for level_handler in &mut levels_handler { + for pending_task_id in &level_handler.pending_tasks_ids() { + level_handler.remove_task(*pending_task_id); + } + } + + // rebuild selector + selector = SpaceReclaimCompactionSelector::default(); + // cut range [3,4] [6] [8,9,10] + levels.member_table_ids = vec![2, 5, 7]; + let expect_task_file_count = vec![2, 1, 4]; + let expect_task_sst_id_range = vec![vec![3, 4], vec![6], vec![7, 8, 9, 10]]; + for (index, x) in expect_task_file_count.iter().enumerate() { + if index == expect_task_file_count.len() - 1 { + levels.member_table_ids = vec![2, 5]; + } + + // // pick space reclaim + let task = selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handler, + &mut local_stats, + HashMap::default(), + ) + .unwrap(); + + assert_compaction_task(&task, &levels_handler); + assert_eq!(task.input.input_levels.len(), 2); + assert_eq!(task.input.input_levels[0].level_idx, 4); + + assert_eq!(task.input.input_levels[0].table_infos.len(), *x); + let select_sst = &task.input.input_levels[0] + .table_infos + .iter() + .map(|sst| sst.id) + .collect_vec(); + assert!(select_sst.is_sorted()); + assert_eq!(expect_task_sst_id_range[index], *select_sst); + + assert_eq!(task.input.input_levels[1].level_idx, 4); + assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.target_level, 4); + assert!(matches!( + task.compaction_task_type, + compact_task::TaskType::SpaceReclaim + )); + } + } } } diff --git a/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs index d04a934a7cdaa..d9cf605da7e04 100644 --- a/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs @@ -12,28 +12,101 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::{HashMap, HashSet}; + +use risingwave_common::catalog::TableOption; +use risingwave_common::constants::hummock::TABLE_OPTION_DUMMY_RETENTION_SECOND; +use risingwave_common::util::epoch::Epoch; +use risingwave_hummock_sdk::compaction_group::StateTableId; +use risingwave_hummock_sdk::key_range::KeyRangeCommon; use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::InputLevel; +use risingwave_pb::hummock::{InputLevel, KeyRange, SstableInfo}; use crate::hummock::compaction::CompactionInput; use crate::hummock::level_handler::LevelHandler; +const MIN_TTL_EXPIRE_INTERVAL_MS: u64 = 60 * 60 * 1000; // 1h + #[derive(Default)] pub struct TtlPickerState { - last_select_index: usize, + // Because of the right_exclusive, we use KeyRangeCommon to determine if the end_bounds + // overlap instead of directly comparing Vec. We don't need to use the start_bound in the + // filter, set it to -inf + + // record the end_bound that has been scanned + pub last_select_end_bound: KeyRange, + + // record the end_bound in the current round of scanning tasks + pub end_bound_in_round: KeyRange, +} + +impl TtlPickerState { + pub fn valid(&self) -> bool { + !self.end_bound_in_round.right.is_empty() + } + + pub fn init(&mut self, key_range: KeyRange) { + self.last_select_end_bound = KeyRange { + left: vec![], + right: key_range.left.clone(), + right_exclusive: true, + }; + self.end_bound_in_round = key_range; + } + + pub fn clear(&mut self) { + self.end_bound_in_round = KeyRange::default(); + self.last_select_end_bound = KeyRange::default(); + } } pub struct TtlReclaimCompactionPicker { max_ttl_reclaim_bytes: u64, - // todo: filter table option + table_id_to_ttl: HashMap, } impl TtlReclaimCompactionPicker { - pub fn new(max_ttl_reclaim_bytes: u64) -> Self { + pub fn new( + max_ttl_reclaim_bytes: u64, + table_id_to_options: HashMap, + ) -> Self { + let table_id_to_ttl: HashMap = table_id_to_options + .iter() + .filter(|id_to_option| { + let table_option = id_to_option.1; + table_option.retention_seconds.is_some() + }) + .map(|id_to_option| (*id_to_option.0, id_to_option.1.retention_seconds.unwrap())) + .collect(); + Self { max_ttl_reclaim_bytes, + table_id_to_ttl, } } + + fn filter(&self, sst: &SstableInfo, current_epoch_time: u64) -> bool { + let table_id_in_sst = sst.table_ids.iter().cloned().collect::>(); + let expire_epoch = + Epoch::from_physical_time(current_epoch_time - MIN_TTL_EXPIRE_INTERVAL_MS); + + for table_id in table_id_in_sst { + match self.table_id_to_ttl.get(&table_id) { + Some(ttl_second_u32) => { + assert!(*ttl_second_u32 != TABLE_OPTION_DUMMY_RETENTION_SECOND); + // default to zero. + let ttl_mill = (*ttl_second_u32 * 1000) as u64; + let min_epoch = expire_epoch.subtract_ms(ttl_mill); + if Epoch(sst.min_epoch) <= min_epoch { + return false; + } + } + None => continue, + } + } + + true + } } impl TtlReclaimCompactionPicker { @@ -48,30 +121,83 @@ impl TtlReclaimCompactionPicker { let mut select_input_ssts = vec![]; let level_handler = &level_handlers[reclaimed_level.level_idx as usize]; - if state.last_select_index >= reclaimed_level.table_infos.len() { - state.last_select_index = 0; + if reclaimed_level.table_infos.is_empty() { + // 1. not file to be picked + state.clear(); + return None; } - let start_indedx = state.last_select_index; + if state.valid() + && state + .last_select_end_bound + .compare_right_with(&state.end_bound_in_round.right) + == std::cmp::Ordering::Greater + { + // in round but end_key overflow + // turn to next_round + state.clear(); + return None; + } + + if !state.valid() { + // new round init key_range bound with table_infos + let first_sst = reclaimed_level.table_infos.first().unwrap(); + let last_sst = reclaimed_level.table_infos.last().unwrap(); + + let key_range_this_round = KeyRange { + left: first_sst.key_range.as_ref().unwrap().left.clone(), + right: last_sst.key_range.as_ref().unwrap().right.clone(), + right_exclusive: last_sst.key_range.as_ref().unwrap().right_exclusive, + }; + + state.init(key_range_this_round); + } + + let current_epoch_time = Epoch::now().0; let mut select_file_size = 0; - for sst in &reclaimed_level.table_infos[start_indedx..] { - state.last_select_index += 1; - if level_handler.is_pending_compact(&sst.id) { + for sst in &reclaimed_level.table_infos { + let unmatched_sst = sst + .key_range + .as_ref() + .unwrap() + .sstable_overlap(&state.last_select_end_bound); + + if unmatched_sst + || level_handler.is_pending_compact(&sst.id) + || self.filter(sst, current_epoch_time) + { + if !select_input_ssts.is_empty() { + // Our goal is to pick as many complete layers of data as possible and keep the + // picked files contiguous to avoid overlapping key_ranges, so the strategy is + // to pick as many contiguous files as possible (at least one) + break; + } + continue; } select_input_ssts.push(sst.clone()); select_file_size += sst.file_size; + if select_file_size > self.max_ttl_reclaim_bytes { break; } } + // turn to next_round if select_input_ssts.is_empty() { + state.clear(); return None; } + let select_last_sst = select_input_ssts.last().unwrap(); + state.last_select_end_bound.full_key_extend(&KeyRange { + left: vec![], + right: select_last_sst.key_range.as_ref().unwrap().right.clone(), + right_exclusive: select_last_sst.key_range.as_ref().unwrap().right_exclusive, + }); + Some(CompactionInput { input_levels: vec![ InputLevel { @@ -93,7 +219,6 @@ impl TtlReclaimCompactionPicker { #[cfg(test)] mod test { - use itertools::Itertools; use risingwave_pb::hummock::compact_task; pub use risingwave_pb::hummock::{KeyRange, Level, LevelType}; @@ -102,7 +227,7 @@ mod test { use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; use crate::hummock::compaction::level_selector::tests::{ assert_compaction_task, generate_l0_nonoverlapping_sublevels, generate_level, - generate_table, + generate_table_with_ids_and_epochs, }; use crate::hummock::compaction::level_selector::{LevelSelector, TtlCompactionSelector}; use crate::hummock::compaction::LocalSelectorStatistic; @@ -117,34 +242,129 @@ mod test { let group_config = CompactionGroup::new(1, config); let l0 = generate_l0_nonoverlapping_sublevels(vec![]); assert_eq!(l0.sub_levels.len(), 0); - let levels = vec![ + + let current_epoch_time = Epoch::now().0; + let expired_epoch = Epoch::from_physical_time( + current_epoch_time - MIN_TTL_EXPIRE_INTERVAL_MS - (1000 * 1000), + ) + .0; + let mut levels = vec![ generate_level(1, vec![]), generate_level(2, vec![]), generate_level( 3, vec![ - generate_table(0, 1, 150, 151, 1), - generate_table(1, 1, 250, 251, 1), + generate_table_with_ids_and_epochs(0, 1, 150, 151, 1, vec![0], 0, 0), + generate_table_with_ids_and_epochs(1, 1, 250, 251, 1, vec![1], 0, 0), ], ), Level { level_idx: 4, level_type: LevelType::Nonoverlapping as i32, table_infos: vec![ - generate_table(2, 1, 0, 100, 1), - generate_table(3, 1, 101, 200, 1), - generate_table(4, 1, 222, 300, 1), - generate_table(5, 1, 333, 400, 1), - generate_table(6, 1, 444, 500, 1), - generate_table(7, 1, 555, 600, 1), - generate_table(8, 1, 666, 700, 1), - generate_table(9, 1, 777, 800, 1), - generate_table(10, 1, 888, 900, 1), + generate_table_with_ids_and_epochs(2, 1, 0, 100, 1, vec![2], expired_epoch, 0), + generate_table_with_ids_and_epochs( + 3, + 1, + 101, + 200, + 1, + vec![3], + expired_epoch, + 0, + ), + generate_table_with_ids_and_epochs( + 4, + 1, + 222, + 300, + 1, + vec![4], + expired_epoch, + u64::MAX, + ), + generate_table_with_ids_and_epochs( + 5, + 1, + 333, + 400, + 1, + vec![5], + expired_epoch, + u64::MAX, + ), + generate_table_with_ids_and_epochs( + 6, + 1, + 444, + 500, + 1, + vec![6], + expired_epoch, + u64::MAX, + ), + generate_table_with_ids_and_epochs( + 7, + 1, + 555, + 600, + 1, + vec![7], + expired_epoch, + u64::MAX, + ), + generate_table_with_ids_and_epochs( + 8, + 1, + 666, + 700, + 1, + vec![8], + expired_epoch, + u64::MAX, + ), + generate_table_with_ids_and_epochs( + 9, + 1, + 777, + 800, + 1, + vec![9], + expired_epoch, + u64::MAX, + ), + generate_table_with_ids_and_epochs( + 10, + 1, + 888, + 1600, + 1, + vec![10], + expired_epoch, + u64::MAX, + ), + generate_table_with_ids_and_epochs( + 11, + 1, + 1600, + 1800, + 1, + vec![10], + expired_epoch, + u64::MAX, + ), ], total_file_size: 0, sub_level_id: 0, }, ]; + + { + let sst_10 = levels[3].table_infos.get_mut(8).unwrap(); + assert_eq!(10, sst_10.id); + sst_10.key_range.as_mut().unwrap().right_exclusive = true; + } + assert_eq!(levels.len(), 4); let levels = Levels { levels, @@ -155,7 +375,17 @@ mod test { let mut local_stats = LocalSelectorStatistic::default(); let mut selector = TtlCompactionSelector::default(); { - // pick space reclaim + let table_id_to_options: HashMap = (2..=10) + .map(|table_id| { + ( + table_id as u32, + TableOption { + retention_seconds: Some(5_u32), + }, + ) + }) + .collect(); + // pick ttl reclaim let task = selector .pick_compaction( 1, @@ -163,6 +393,7 @@ mod test { &levels, &mut levels_handler, &mut local_stats, + table_id_to_options, ) .unwrap(); assert_compaction_task(&task, &levels_handler); @@ -192,7 +423,18 @@ mod test { } } - // pick space reclaim + let table_id_to_options: HashMap = (2..=10) + .map(|table_id| { + ( + table_id as u32, + TableOption { + retention_seconds: Some(5_u32), + }, + ) + }) + .collect(); + + // pick ttl reclaim let task = selector .pick_compaction( 1, @@ -200,18 +442,15 @@ mod test { &levels, &mut levels_handler, &mut local_stats, + table_id_to_options.clone(), ) .unwrap(); assert_compaction_task(&task, &levels_handler); assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 4); - // test select index, picker will select file from last_select_index - let all_file_count = levels.get_levels().last().unwrap().get_table_infos().len(); - assert_eq!( - task.input.input_levels[0].table_infos.len(), - all_file_count - 5 - ); + // test select index, picker will select file from state + assert_eq!(task.input.input_levels[0].table_infos.len(), 4); let mut start_id = 7; for sst in &task.input.input_levels[0].table_infos { @@ -226,6 +465,299 @@ mod test { task.compaction_task_type, compact_task::TaskType::Ttl )); + + // test pick key_range right exclusive + let task = selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handler, + &mut local_stats, + table_id_to_options.clone(), + ) + .unwrap(); + assert_compaction_task(&task, &levels_handler); + assert_eq!(task.input.input_levels.len(), 2); + assert_eq!(task.input.input_levels[0].level_idx, 4); + assert_eq!(task.input.input_levels[0].table_infos.len(), 1); + assert_eq!(task.input.input_levels[1].level_idx, 4); + assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.target_level, 4); + assert!(matches!( + task.compaction_task_type, + compact_task::TaskType::Ttl + )); + for sst in &task.input.input_levels[0].table_infos { + assert_eq!(start_id, sst.id); + start_id += 1; + } + + assert!(selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handler, + &mut local_stats, + table_id_to_options, + ) + .is_none()) + } + + { + for level_handler in &mut levels_handler { + for pending_task_id in &level_handler.pending_tasks_ids() { + level_handler.remove_task(*pending_task_id); + } + } + + // rebuild selector + selector = TtlCompactionSelector::default(); + let mut table_id_to_options: HashMap = (2..=10) + .map(|table_id| { + ( + table_id as u32, + TableOption { + retention_seconds: Some(7200), + }, + ) + }) + .collect(); + + table_id_to_options.insert( + 5, + TableOption { + retention_seconds: Some(5), + }, + ); + + // // pick ttl reclaim + let task = selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handler, + &mut local_stats, + table_id_to_options, + ) + .unwrap(); + assert_compaction_task(&task, &levels_handler); + assert_eq!(task.input.input_levels.len(), 2); + assert_eq!(task.input.input_levels[0].level_idx, 4); + + // test table_option_filter + assert_eq!(task.input.input_levels[0].table_infos.len(), 1); + let select_sst = &task.input.input_levels[0].table_infos.first().unwrap(); + assert_eq!(select_sst.id, 5); + + assert_eq!(task.input.input_levels[1].level_idx, 4); + assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.target_level, 4); + assert!(matches!( + task.compaction_task_type, + compact_task::TaskType::Ttl + )); + } + + { + // test empty table_option filter + + for level_handler in &mut levels_handler { + for pending_task_id in &level_handler.pending_tasks_ids() { + level_handler.remove_task(*pending_task_id); + } + } + + // rebuild selector + selector = TtlCompactionSelector::default(); + + // // pick ttl reclaim + let task = selector.pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handler, + &mut local_stats, + HashMap::default(), + ); + + // empty table_options does not select any files + assert!(task.is_none()); + } + + { + // test continuous file selection + for level_handler in &mut levels_handler { + for pending_task_id in &level_handler.pending_tasks_ids() { + level_handler.remove_task(*pending_task_id); + } + } + + // rebuild selector + selector = TtlCompactionSelector::default(); + let mut table_id_to_options: HashMap = (2..=10) + .map(|table_id| { + ( + table_id as u32, + TableOption { + retention_seconds: Some(5_u32), + }, + ) + }) + .collect(); + + // cut range [2,3,4] [6,7] [10] + table_id_to_options.insert( + 5, + TableOption { + retention_seconds: Some(7200_u32), + }, + ); + + table_id_to_options.insert( + 8, + TableOption { + retention_seconds: Some(7200_u32), + }, + ); + + table_id_to_options.insert( + 9, + TableOption { + retention_seconds: Some(7200_u32), + }, + ); + + let expect_task_file_count = vec![3, 2, 1]; + let expect_task_sst_id_range = vec![vec![2, 3, 4], vec![6, 7], vec![10]]; + for (index, x) in expect_task_file_count.iter().enumerate() { + // // pick ttl reclaim + let task = selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handler, + &mut local_stats, + table_id_to_options.clone(), + ) + .unwrap(); + + assert_compaction_task(&task, &levels_handler); + assert_eq!(task.input.input_levels.len(), 2); + assert_eq!(task.input.input_levels[0].level_idx, 4); + + // test table_option_filter + assert_eq!(task.input.input_levels[0].table_infos.len(), *x); + let select_sst = &task.input.input_levels[0] + .table_infos + .iter() + .map(|sst| sst.id) + .collect_vec(); + assert!(select_sst.is_sorted()); + assert_eq!(expect_task_sst_id_range[index], *select_sst); + + assert_eq!(task.input.input_levels[1].level_idx, 4); + assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.target_level, 4); + assert!(matches!( + task.compaction_task_type, + compact_task::TaskType::Ttl + )); + } + } + + { + // test continuous file selection with filter change + for level_handler in &mut levels_handler { + for pending_task_id in &level_handler.pending_tasks_ids() { + level_handler.remove_task(*pending_task_id); + } + } + + // rebuild selector + selector = TtlCompactionSelector::default(); + let mut table_id_to_options: HashMap = (2..=10) + .map(|table_id| { + ( + table_id as u32, + TableOption { + retention_seconds: Some(5_u32), + }, + ) + }) + .collect(); + + // cut range [2,3,4] [6,7] [10] + table_id_to_options.insert( + 5, + TableOption { + retention_seconds: Some(7200_u32), + }, + ); + + table_id_to_options.insert( + 8, + TableOption { + retention_seconds: Some(7200_u32), + }, + ); + + table_id_to_options.insert( + 9, + TableOption { + retention_seconds: Some(7200_u32), + }, + ); + + let expect_task_file_count = vec![3, 3]; + let expect_task_sst_id_range = vec![vec![2, 3, 4], vec![5, 6, 7]]; + for (index, x) in expect_task_file_count.iter().enumerate() { + if index == expect_task_file_count.len() - 1 { + table_id_to_options.insert( + 5, + TableOption { + retention_seconds: Some(5_u32), + }, + ); + } + + // // pick ttl reclaim + let task = selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handler, + &mut local_stats, + table_id_to_options.clone(), + ) + .unwrap(); + + assert_compaction_task(&task, &levels_handler); + assert_eq!(task.input.input_levels.len(), 2); + assert_eq!(task.input.input_levels[0].level_idx, 4); + + // test table_option_filter + assert_eq!(task.input.input_levels[0].table_infos.len(), *x); + let select_sst = &task.input.input_levels[0] + .table_infos + .iter() + .map(|sst| sst.id) + .collect_vec(); + assert!(select_sst.is_sorted()); + assert_eq!(expect_task_sst_id_range[index], *select_sst); + + assert_eq!(task.input.input_levels[1].level_idx, 4); + assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.target_level, 4); + assert!(matches!( + task.compaction_task_type, + compact_task::TaskType::Ttl + )); + } } } } diff --git a/src/meta/src/hummock/compaction_schedule_policy.rs b/src/meta/src/hummock/compaction_schedule_policy.rs index f49f88e31b38b..e5f191a6f3ca4 100644 --- a/src/meta/src/hummock/compaction_schedule_policy.rs +++ b/src/meta/src/hummock/compaction_schedule_policy.rs @@ -422,6 +422,8 @@ mod tests { stale_key_count: 0, total_key_count: 0, divide_version: 0, + min_epoch: 0, + max_epoch: 0, }], }], splits: vec![], diff --git a/src/meta/src/hummock/compaction_scheduler.rs b/src/meta/src/hummock/compaction_scheduler.rs index 0beacea7168e6..4b01b64130ed6 100644 --- a/src/meta/src/hummock/compaction_scheduler.rs +++ b/src/meta/src/hummock/compaction_scheduler.rs @@ -16,6 +16,9 @@ use std::collections::{HashMap, HashSet}; use std::sync::Arc; use std::time::Duration; +use futures::future::{Either, Shared}; +use futures::stream::select; +use futures::{FutureExt, Stream, StreamExt}; use parking_lot::Mutex; use risingwave_hummock_sdk::compact::compact_task_to_string; use risingwave_hummock_sdk::CompactionGroupId; @@ -23,9 +26,10 @@ use risingwave_pb::hummock::compact_task::{self, TaskStatus}; use risingwave_pb::hummock::subscribe_compact_tasks_response::Task; use risingwave_pb::hummock::CompactTask; use tokio::sync::mpsc::error::SendError; -use tokio::sync::mpsc::UnboundedSender; +use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::sync::oneshot::Receiver; use tokio::sync::Notify; +use tokio_stream::wrappers::{IntervalStream, UnboundedReceiverStream}; use super::Compactor; use crate::hummock::compaction::{ @@ -45,7 +49,7 @@ type CompactionRequestChannelItem = (CompactionGroupId, compact_task::TaskType); /// compaction groups. pub struct CompactionRequestChannel { request_tx: UnboundedSender, - scheduled: Mutex>, + scheduled: Mutex>, } #[derive(Debug, PartialEq)] @@ -72,16 +76,21 @@ impl CompactionRequestChannel { task_type: compact_task::TaskType, ) -> Result> { let mut guard = self.scheduled.lock(); - if guard.contains(&compaction_group) { + let key = (compaction_group, task_type); + if guard.contains(&key) { return Ok(false); } - self.request_tx.send((compaction_group, task_type))?; - guard.insert(compaction_group); + self.request_tx.send(key)?; + guard.insert(key); Ok(true) } - pub fn unschedule(&self, compaction_group: CompactionGroupId) { - self.scheduled.lock().remove(&compaction_group); + pub fn unschedule( + &self, + compaction_group: CompactionGroupId, + task_type: compact_task::TaskType, + ) { + self.scheduled.lock().remove(&(compaction_group, task_type)); } } @@ -118,8 +127,8 @@ where } } - pub async fn start(&self, mut shutdown_rx: Receiver<()>) { - let (sched_tx, mut sched_rx) = + pub async fn start(&self, shutdown_rx: Receiver<()>) { + let (sched_tx, sched_rx) = tokio::sync::mpsc::unbounded_channel::(); let sched_channel = Arc::new(CompactionRequestChannel::new(sched_tx)); @@ -129,86 +138,22 @@ where ); tracing::info!("Start compaction scheduler."); - let mut min_trigger_interval = tokio::time::interval(Duration::from_secs( - self.env.opts.periodic_compaction_interval_sec, - )); - min_trigger_interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); - let mut min_space_reclaim_trigger_interval = tokio::time::interval(Duration::from_secs( + let compaction_selectors = Self::init_selectors(); + let shutdown_rx = shutdown_rx.shared(); + let schedule_event_stream = Self::scheduler_event_stream( + sched_rx, self.env.opts.periodic_space_reclaim_compaction_interval_sec, - )); - min_space_reclaim_trigger_interval - .set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); - let mut compaction_selectors = Self::init_selectors(); - - loop { - let (compaction_group, task_type) = tokio::select! { - recv = sched_rx.recv() => { - match recv { - Some((compaction_group, task_type)) => (compaction_group, task_type), - None => { - tracing::warn!("Compactor Scheduler: The Hummock manager has dropped the connection, - it means it has either died or started a new session. Exiting."); - return; - } - } - }, - - _ = min_trigger_interval.tick() => { - // Disable periodic trigger for compaction_deterministic_test. - if self.env.opts.compaction_deterministic_test { - continue; - } - // Periodically trigger compaction for all compaction groups. - for cg_id in self.hummock_manager.compaction_group_ids().await { - if let Err(e) = sched_channel.try_sched_compaction(cg_id, compact_task::TaskType::Dynamic) { - tracing::warn!("Failed to schedule base compaction for compaction group {}. {}", cg_id, e); - } - } - continue; - }, - - _ = min_space_reclaim_trigger_interval.tick() => { - // Disable periodic trigger for compaction_deterministic_test. - if self.env.opts.compaction_deterministic_test { - continue; - } - // Periodically trigger space_reclaim compaction for all compaction groups. - for cg_id in self.hummock_manager.compaction_group_ids().await { - if let Err(e) = sched_channel.try_sched_compaction(cg_id, compact_task::TaskType::SpaceReclaim) { - tracing::warn!("Failed to schedule base compaction for compaction group {}. {}", cg_id, e); - } - } - continue; - } - - // Shutdown compactor scheduler - _ = &mut shutdown_rx => { - break; - } - }; - - sync_point::sync_point!("BEFORE_SCHEDULE_COMPACTION_TASK"); - sched_channel.unschedule(compaction_group); - - // Wait for a compactor to become available. - let compactor = loop { - if let Some(compactor) = self.hummock_manager.get_idle_compactor().await { - break compactor; - } else { - tracing::debug!("No available compactor, pausing compaction."); - tokio::select! { - _ = self.compaction_resume_notifier.notified() => {}, - _ = &mut shutdown_rx => { - return; - } - } - } - }; - let selector = compaction_selectors.get_mut(&task_type).unwrap(); - self.pick_and_assign(compaction_group, compactor, sched_channel.clone(), selector) - .await; - } + self.env.opts.periodic_ttl_reclaim_compaction_interval_sec, + self.env.opts.periodic_compaction_interval_sec, + ); + self.schedule_loop( + sched_channel.clone(), + shutdown_rx, + compaction_selectors, + schedule_event_stream, + ) + .await; } fn init_selectors() -> HashMap> { @@ -318,7 +263,12 @@ where ); } Err(err) => { - tracing::warn!("Failed to assign compaction task to compactor: {:#?}", err); + tracing::warn!( + "Failed to assign {:?} compaction task to compactor {} : {:#?}", + compact_task.task_type().as_str_name(), + compactor.context_id(), + err + ); match err { Error::CompactionTaskAlreadyAssigned(_, _) => { panic!("Compaction scheduler is the only tokio task that can assign task."); @@ -368,6 +318,207 @@ where } ScheduleStatus::Ok } + + async fn schedule_loop( + &self, + sched_channel: Arc, + shutdown_rx: Shared>, + mut compaction_selectors: HashMap>, + event_stream: impl Stream, + ) { + use futures::pin_mut; + pin_mut!(event_stream); + + loop { + let item = futures::future::select(event_stream.next(), shutdown_rx.clone()).await; + match item { + Either::Left((event, _)) => { + if let Some(event) = event { + match event { + SchedulerEvent::Channel((compaction_group, task_type)) => { + // recv + if !self + .on_handle_compact( + compaction_group, + &mut compaction_selectors, + task_type, + sched_channel.clone(), + shutdown_rx.clone(), + ) + .await + { + break; + } + } + SchedulerEvent::DynamicTrigger => { + // Disable periodic trigger for compaction_deterministic_test. + if self.env.opts.compaction_deterministic_test { + continue; + } + // Periodically trigger compaction for all compaction groups. + self.on_handle_trigger_multi_grouop( + sched_channel.clone(), + compact_task::TaskType::Dynamic, + ) + .await; + continue; + } + SchedulerEvent::SpaceReclaimTrigger => { + // Disable periodic trigger for compaction_deterministic_test. + if self.env.opts.compaction_deterministic_test { + continue; + } + // Periodically trigger compaction for all compaction groups. + self.on_handle_trigger_multi_grouop( + sched_channel.clone(), + compact_task::TaskType::SpaceReclaim, + ) + .await; + continue; + } + SchedulerEvent::TtlReclaimTrigger => { + // Disable periodic trigger for compaction_deterministic_test. + if self.env.opts.compaction_deterministic_test { + continue; + } + // Periodically trigger compaction for all compaction groups. + self.on_handle_trigger_multi_grouop( + sched_channel.clone(), + compact_task::TaskType::Ttl, + ) + .await; + continue; + } + } + } + } + + Either::Right((_, _shutdown)) => { + break; + } + } + } + } + + async fn on_handle_compact( + &self, + compaction_group: CompactionGroupId, + compaction_selectors: &mut HashMap>, + task_type: compact_task::TaskType, + sched_channel: Arc, + shutdown_rx: Shared>, + ) -> bool { + sync_point::sync_point!("BEFORE_SCHEDULE_COMPACTION_TASK"); + sched_channel.unschedule(compaction_group, task_type); + + self.task_dispatch( + compaction_group, + task_type, + compaction_selectors, + sched_channel, + shutdown_rx, + ) + .await + } + + async fn on_handle_trigger_multi_grouop( + &self, + sched_channel: Arc, + task_type: compact_task::TaskType, + ) { + for cg_id in self.hummock_manager.compaction_group_ids().await { + if let Err(e) = sched_channel.try_sched_compaction(cg_id, task_type) { + tracing::warn!( + "Failed to schedule {:?} compaction for compaction group {}. {}", + task_type, + cg_id, + e + ); + } + } + } + + async fn task_dispatch( + &self, + compaction_group: CompactionGroupId, + task_type: compact_task::TaskType, + compaction_selectors: &mut HashMap>, + sched_channel: Arc, + mut shutdown_rx: Shared>, + ) -> bool { + // Wait for a compactor to become available. + let compactor = loop { + if let Some(compactor) = self.hummock_manager.get_idle_compactor().await { + break compactor; + } else { + tracing::debug!("No available compactor, pausing compaction."); + tokio::select! { + _ = self.compaction_resume_notifier.notified() => {}, + _ = &mut shutdown_rx => { + return false; + } + } + } + }; + let selector = compaction_selectors.get_mut(&task_type).unwrap(); + self.pick_and_assign(compaction_group, compactor, sched_channel.clone(), selector) + .await; + + true + } +} + +enum SchedulerEvent { + Channel((CompactionGroupId, compact_task::TaskType)), + DynamicTrigger, + SpaceReclaimTrigger, + TtlReclaimTrigger, +} + +impl CompactionScheduler +where + S: MetaStore, +{ + fn scheduler_event_stream( + sched_rx: UnboundedReceiver<(CompactionGroupId, compact_task::TaskType)>, + periodic_space_reclaim_compaction_interval_sec: u64, + periodic_ttl_reclaim_compaction_interval_sec: u64, + periodic_compaction_interval_sec: u64, + ) -> impl Stream { + let dynamic_channel_trigger = + UnboundedReceiverStream::new(sched_rx).map(SchedulerEvent::Channel); + + let mut min_trigger_interval = + tokio::time::interval(Duration::from_secs(periodic_compaction_interval_sec)); + min_trigger_interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + let dynamic_tick_trigger = + IntervalStream::new(min_trigger_interval).map(|_| SchedulerEvent::DynamicTrigger); + + let mut min_space_reclaim_trigger_interval = tokio::time::interval(Duration::from_secs( + periodic_space_reclaim_compaction_interval_sec, + )); + + min_space_reclaim_trigger_interval + .set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + let space_reclaim_trigger = IntervalStream::new(min_space_reclaim_trigger_interval) + .map(|_| SchedulerEvent::SpaceReclaimTrigger); + + let mut min_ttl_reclaim_trigger_interval = tokio::time::interval(Duration::from_secs( + periodic_ttl_reclaim_compaction_interval_sec, + )); + min_ttl_reclaim_trigger_interval + .set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + let ttl_reclaim_trigger = IntervalStream::new(min_ttl_reclaim_trigger_interval) + .map(|_| SchedulerEvent::TtlReclaimTrigger); + + select( + dynamic_channel_trigger, + select( + dynamic_tick_trigger, + select(space_reclaim_trigger, ttl_reclaim_trigger), + ), + ) + } } #[cfg(test)] @@ -584,16 +735,15 @@ mod tests { fail::remove(fp_cancel_compact_task); assert_eq!(hummock_manager.list_all_tasks_ids().await.len(), 1); // Notified to retry cancellation. - let mut task_to_cancel = match rx.recv().await.unwrap() { - LocalNotification::WorkerNodeIsDeleted(_) => { - panic!() - } - LocalNotification::CompactionTaskNeedCancel(task_to_cancel) => task_to_cancel, + if let LocalNotification::CompactionTaskNeedCancel(mut task_to_cancel) = + rx.recv().await.unwrap() + { + hummock_manager + .cancel_compact_task(&mut task_to_cancel, TaskStatus::ManualCanceled) + .await + .unwrap(); }; - hummock_manager - .cancel_compact_task(&mut task_to_cancel, TaskStatus::ManualCanceled) - .await - .unwrap(); + assert!(hummock_manager.list_all_tasks_ids().await.is_empty()); // Succeeded. diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 73faf60da8f9d..07fd5c8889e6f 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -44,6 +44,7 @@ use risingwave_pb::hummock::{ version_update_payload, CompactTask, CompactTaskAssignment, CompactionConfig, GroupDelta, HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, HummockVersion, HummockVersionDelta, HummockVersionDeltas, HummockVersionStats, IntraLevelDelta, LevelType, + TableOption, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use tokio::sync::oneshot::Sender; @@ -732,6 +733,11 @@ where compaction_group_id: CompactionGroupId, selector: &mut Box, ) -> Result> { + // TODO: `get_all_table_options` will hold catalog_manager async lock, to avoid holding the + // lock in compaction_guard, take out all table_options in advance there may be a + // waste of resources here, need to add a more efficient filter in catalog_manager + let all_table_id_to_option = self.catalog_manager.get_all_table_options().await; + let mut compaction_guard = write_lock!(self, compaction).await; let compaction = compaction_guard.deref_mut(); let compaction_statuses = &mut compaction.compaction_statuses; @@ -777,12 +783,21 @@ where let can_trivial_move = matches!(selector.task_type(), compact_task::TaskType::Dynamic); let mut stats = LocalSelectorStatistic::default(); + let member_table_ids = ¤t_version + .get_compaction_group_levels(compaction_group_id) + .member_table_ids; + let table_id_to_option: HashMap = all_table_id_to_option + .into_iter() + .filter(|(table_id, _)| member_table_ids.contains(table_id)) + .collect(); + let compact_task = compact_status.get_compact_task( current_version.get_compaction_group_levels(compaction_group_id), task_id as HummockCompactionTaskId, &group_config, &mut stats, selector, + table_id_to_option.clone(), ); stats.report_to_metrics(compaction_group_id, self.metrics.as_ref()); let mut compact_task = match compact_task { @@ -828,12 +843,15 @@ where } } - compact_task.table_options = self - .catalog_manager - .get_table_options(&compact_task.existing_table_ids) - .await - .iter() - .map(|(k, v)| (*k, v.into())) + compact_task.table_options = table_id_to_option + .into_iter() + .filter_map(|(table_id, table_option)| { + if compact_task.existing_table_ids.contains(&table_id) { + return Some((table_id, TableOption::from(&table_option))); + } + + None + }) .collect(); compact_task.current_epoch_time = Epoch::now().0; compact_task.compaction_filter_mask = @@ -1266,9 +1284,10 @@ where if !deterministic_mode && matches!(compact_task.task_type(), compact_task::TaskType::Dynamic) { + // only try send Dynamic compaction self.try_send_compaction_request( compact_task.compaction_group_id, - compact_task.task_type(), + compact_task::TaskType::Dynamic, ); } diff --git a/src/meta/src/hummock/manager/worker.rs b/src/meta/src/hummock/manager/worker.rs index a4d1031320c04..1d8abda94c21a 100644 --- a/src/meta/src/hummock/manager/worker.rs +++ b/src/meta/src/hummock/manager/worker.rs @@ -150,6 +150,7 @@ where tracing::info!("Cancelled compaction task {}", task_id); sync_point!("AFTER_CANCEL_COMPACTION_TASK_ASYNC"); } + LocalNotification::SystemParamsChange(_) => {} } } } diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index c7b7c279aa05e..4b5327f5e0d81 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -202,7 +202,7 @@ impl HummockMetaClient for MockHummockMetaClient { let (task_tx, task_rx) = tokio::sync::mpsc::unbounded_channel(); let handle = tokio::spawn(async move { while let Some((group, task_type)) = sched_rx.recv().await { - sched_channel.unschedule(group); + sched_channel.unschedule(group, task_type); let mut selector: Box = match task_type { compact_task::TaskType::Dynamic => default_level_selector(), diff --git a/src/meta/src/hummock/mod.rs b/src/meta/src/hummock/mod.rs index 8801012eb27ca..77c5854cc2ad0 100644 --- a/src/meta/src/hummock/mod.rs +++ b/src/meta/src/hummock/mod.rs @@ -11,7 +11,6 @@ // 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. - pub mod compaction; mod compaction_schedule_policy; mod compaction_scheduler; diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 0a74ded9267b4..8400d0a74308a 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -163,6 +163,8 @@ pub fn generate_test_tables(epoch: u64, sst_ids: Vec) -> Vec Pin + Send>> { MetaBackend::Mem => MetaStoreBackend::Mem, }; + validate_config(&config); + let max_heartbeat_interval = Duration::from_secs(config.meta.max_heartbeat_interval_secs as u64); let barrier_interval = Duration::from_millis(config.streaming.barrier_interval_ms as u64); @@ -248,6 +251,9 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { periodic_space_reclaim_compaction_interval_sec: config .meta .periodic_space_reclaim_compaction_interval_sec, + periodic_ttl_reclaim_compaction_interval_sec: config + .meta + .periodic_ttl_reclaim_compaction_interval_sec, }, ) .await @@ -267,3 +273,11 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { } }) } + +fn validate_config(config: &RwConfig) { + if config.meta.meta_leader_lease_secs <= 1 { + let error_msg = "meta leader lease secs should be larger than 1"; + tracing::error!(error_msg); + panic!("{}", error_msg); + } +} diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index 014b8f641b58b..f720d6f0dc1d5 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -200,15 +200,10 @@ impl DatabaseManager { self.tables.values().cloned().collect_vec() } - pub fn get_table_options(&self, table_ids: &[TableId]) -> HashMap { + pub fn get_all_table_options(&self) -> HashMap { self.tables .iter() - .filter_map(|(id, table)| { - if table_ids.contains(id) { - return Some((*id, TableOption::build_table_option(&table.properties))); - } - None - }) + .map(|(id, table)| (*id, TableOption::build_table_option(&table.properties))) .collect() } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index e9faf9c8441c8..5e5c19e0cea33 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -1459,8 +1459,8 @@ where self.core.lock().await.database.list_tables() } - pub async fn get_table_options(&self, table_ids: &[TableId]) -> HashMap { - self.core.lock().await.database.get_table_options(table_ids) + pub async fn get_all_table_options(&self) -> HashMap { + self.core.lock().await.database.get_all_table_options() } pub async fn list_table_ids(&self, schema_id: SchemaId) -> Vec { diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 6557d5d4b6450..faabe2c827434 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -29,7 +29,6 @@ use tokio::task::JoinHandle; use crate::manager::{IdCategory, LocalNotification, MetaSrvEnv}; use crate::model::{MetadataModel, Worker, INVALID_EXPIRE_AT}; -use crate::rpc::metrics::MetaMetrics; use crate::storage::MetaStore; use crate::{MetaError, MetaResult}; @@ -87,38 +86,8 @@ where self.core.read().await } - pub async fn start_worker_num_monitor( - cluster_manager: ClusterManagerRef, - interval: Duration, - meta_metrics: Arc, - ) -> (JoinHandle<()>, Sender<()>) { - let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel(); - let join_handle = tokio::spawn(async move { - let mut monitor_interval = tokio::time::interval(interval); - monitor_interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); - loop { - tokio::select! { - // Wait for interval - _ = monitor_interval.tick() => {}, - // Shutdown monitor - _ = &mut shutdown_rx => { - tracing::info!("Worker number monitor is stopped"); - return; - } - } - - for (worker_type, worker_num) in - cluster_manager.core.read().await.count_worker_node() - { - meta_metrics - .worker_num - .with_label_values(&[(worker_type.as_str_name())]) - .set(worker_num as i64); - } - } - }); - - (join_handle, shutdown_tx) + pub async fn count_worker_node(&self) -> HashMap { + self.core.read().await.count_worker_node() } /// A worker node will immediately register itself to meta when it bootstraps. diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 631581b962663..1959c895db7ee 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -68,7 +68,7 @@ pub struct MetaOpts { pub max_idle_ms: u64, /// Whether run in compaction detection test mode pub compaction_deterministic_test: bool, - + // TODO: this will be read from system param channel and should be removed to avoid misuse pub checkpoint_frequency: usize, /// Interval of GC metadata in meta store and stale SSTs in object store. @@ -113,6 +113,9 @@ pub struct MetaOpts { /// Schedule space_reclaim_compaction for all compaction groups with this interval. pub periodic_space_reclaim_compaction_interval_sec: u64, + + /// Schedule ttl_reclaim_compaction for all compaction groups with this interval. + pub periodic_ttl_reclaim_compaction_interval_sec: u64, } impl MetaOpts { @@ -141,6 +144,7 @@ impl MetaOpts { state_store: None, data_directory: "hummock_001".to_string(), periodic_space_reclaim_compaction_interval_sec: 60, + periodic_ttl_reclaim_compaction_interval_sec: 60, } } diff --git a/src/meta/src/manager/notification.rs b/src/meta/src/manager/notification.rs index f3d9436b7f86e..35e0dfca6a846 100644 --- a/src/meta/src/manager/notification.rs +++ b/src/meta/src/manager/notification.rs @@ -16,6 +16,7 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; use std::sync::Arc; +use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_pb::common::{WorkerNode, WorkerType}; use risingwave_pb::hummock::CompactTask; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -37,6 +38,7 @@ pub type NotificationVersion = u64; pub enum LocalNotification { WorkerNodeIsDeleted(WorkerNode), CompactionTaskNeedCancel(CompactTask), + SystemParamsChange(SystemParamsReader), } #[derive(Debug)] diff --git a/src/meta/src/manager/streaming_job.rs b/src/meta/src/manager/streaming_job.rs index 7fe1b84ac56bb..9073f6dec93dd 100644 --- a/src/meta/src/manager/streaming_job.rs +++ b/src/meta/src/manager/streaming_job.rs @@ -14,7 +14,7 @@ use std::collections::HashMap; -use risingwave_common::catalog::TableId; +use risingwave_common::catalog::{TableId, TableVersionId}; use risingwave_pb::catalog::{Index, Sink, Source, Table}; use crate::model::FragmentId; @@ -138,6 +138,12 @@ impl StreamingJob { } } + /// Returns the [`TableVersionId`] if this job contains a table. + pub fn table_version_id(&self) -> Option { + self.table() + .map(|t| t.get_version().expect("table must be versioned").version) + } + /// Returns the optional [`Source`] if this is a `Table` streaming job. /// /// Only used for registering sources for creating tables with connectors. diff --git a/src/meta/src/manager/system_param/mod.rs b/src/meta/src/manager/system_param/mod.rs index 119c25d8079e6..c155b27285bfc 100644 --- a/src/meta/src/manager/system_param/mod.rs +++ b/src/meta/src/manager/system_param/mod.rs @@ -70,6 +70,14 @@ impl SystemParamManager { mem_txn.commit(); + // Sync params to other managers on the meta node only once, since it's infallible. + self.env + .notification_manager() + .notify_local_subscribers(super::LocalNotification::SystemParamsChange( + params.clone().into(), + )) + .await; + Ok(()) } diff --git a/src/meta/src/rpc/election_client.rs b/src/meta/src/rpc/election_client.rs index ef62d4239a4c4..b54b53abd4e7b 100644 --- a/src/meta/src/rpc/election_client.rs +++ b/src/meta/src/rpc/election_client.rs @@ -146,15 +146,25 @@ impl ElectionClient for EtcdElectionClient { let mut ticker = time::interval(Duration::from_secs(1)); + // timeout controller, when keep alive fails for more than a certain period of time + // before it is considered a complete failure + let mut timeout = time::interval(Duration::from_secs((ttl / 2) as u64)); + timeout.reset(); + loop { tokio::select! { biased; + _ = timeout.tick() => { + tracing::warn!("lease {} keep alive timeout", lease_id); + keep_alive_fail_tx.send(()).unwrap(); + break; + } + _ = ticker.tick() => { if let Err(err) = keeper.keep_alive().await { - tracing::error!("keep alive for lease {} failed {}", lease_id, err); - keep_alive_fail_tx.send(()).unwrap(); - break; + tracing::warn!("keep alive for lease {} failed {}", lease_id, err); + continue } match resp_stream.message().await { @@ -164,16 +174,23 @@ impl ElectionClient for EtcdElectionClient { keep_alive_fail_tx.send(()).unwrap(); break; } + + timeout.reset(); }, Ok(None) => { tracing::warn!("lease keeper for lease {} response stream closed unexpected", lease_id); - keep_alive_fail_tx.send(()).unwrap(); - break; + + // try to re-create lease keeper, with timeout as ttl / 2 + if let Ok(Ok((keeper_, resp_stream_))) = time::timeout(Duration::from_secs((ttl / 2) as u64), lease_client.keep_alive(lease_id)).await { + keeper = keeper_; + resp_stream = resp_stream_; + }; + + continue; } Err(e) => { tracing::error!("lease keeper failed {}", e.to_string()); - keep_alive_fail_tx.send(()).unwrap(); - break; + continue; } }; } @@ -224,7 +241,15 @@ impl ElectionClient for EtcdElectionClient { }, resp = observe_stream.next() => { match resp { - None => unreachable!(), + None => { + tracing::warn!("observe stream closed unexpected, recreating"); + + // try to re-create observe stream, with timeout as ttl / 2 + if let Ok(Ok(stream)) = time::timeout(Duration::from_secs((ttl / 2) as u64), election_client.observe(META_ELECTION_KEY)).await { + observe_stream = stream; + tracing::info!("recreating observe stream"); + } + } Some(Ok(leader)) => { if let Some(kv) = leader.kv() && kv.value() != self.id.as_bytes() { tracing::warn!("leader has been changed to {}", String::from_utf8_lossy(kv.value()).to_string()); @@ -232,8 +257,8 @@ impl ElectionClient for EtcdElectionClient { } } Some(Err(e)) => { - tracing::error!("error {} received from leader observe stream", e.to_string()); - break; + tracing::warn!("error {} received from leader observe stream", e.to_string()); + continue } } } diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index ea722f6def714..ba95b7d32f8f6 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -13,6 +13,8 @@ // limitations under the License. use std::sync::atomic::AtomicU64; +use std::sync::Arc; +use std::time::Duration; use prometheus::{ exponential_buckets, histogram_opts, register_histogram_vec_with_registry, @@ -20,6 +22,13 @@ use prometheus::{ register_int_gauge_vec_with_registry, register_int_gauge_with_registry, Histogram, HistogramVec, IntCounterVec, IntGauge, IntGaugeVec, Registry, }; +use risingwave_pb::common::WorkerType; +use tokio::sync::oneshot::Sender; +use tokio::task::JoinHandle; + +use crate::manager::ClusterManagerRef; +use crate::rpc::server::ElectionClientRef; +use crate::storage::MetaStore; pub struct MetaMetrics { registry: Registry, @@ -34,7 +43,7 @@ pub struct MetaMetrics { /// Latency between each barrier send pub barrier_send_latency: Histogram, - /// The number of all barriers. It is the sum of barreriers that are in-flight or completed but + /// The number of all barriers. It is the sum of barriers that are in-flight or completed but /// waiting for other barriers pub all_barrier_nums: IntGauge, /// The number of in-flight barriers @@ -81,6 +90,9 @@ pub struct MetaMetrics { /// The number of workers in the cluster. pub worker_num: IntGaugeVec, pub compact_skip_frequency: IntCounterVec, + + /// The roles of all meta nodes in the cluster. + pub meta_type: IntGaugeVec, } impl MetaMetrics { @@ -256,6 +268,14 @@ impl MetaMetrics { ) .unwrap(); + let meta_type = register_int_gauge_vec_with_registry!( + "meta_num", + "role of meta nodes in the cluster", + &["worker_addr", "role"], + registry, + ) + .unwrap(); + Self { registry, @@ -286,6 +306,7 @@ impl MetaMetrics { time_after_last_observation: AtomicU64::new(0), worker_num, + meta_type, } } @@ -298,3 +319,46 @@ impl Default for MetaMetrics { Self::new() } } + +pub async fn start_worker_info_monitor( + cluster_manager: ClusterManagerRef, + election_client: Option, + interval: Duration, + meta_metrics: Arc, +) -> (JoinHandle<()>, Sender<()>) { + let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel(); + let join_handle = tokio::spawn(async move { + let mut monitor_interval = tokio::time::interval(interval); + monitor_interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + loop { + tokio::select! { + // Wait for interval + _ = monitor_interval.tick() => {}, + // Shutdown monitor + _ = &mut shutdown_rx => { + tracing::info!("Worker number monitor is stopped"); + return; + } + } + + for (worker_type, worker_num) in cluster_manager.count_worker_node().await { + meta_metrics + .worker_num + .with_label_values(&[(worker_type.as_str_name())]) + .set(worker_num as i64); + } + if let Some(client) = &election_client && let Ok(meta_members) = client.get_members().await { + meta_metrics + .worker_num + .with_label_values(&[WorkerType::Meta.as_str_name()]) + .set(meta_members.len() as i64); + meta_members.into_iter().for_each(|m| { + let role = if m.is_leader {"leader"} else {"follower"}; + meta_metrics.meta_type.with_label_values(&[&m.id, role]).set(1); + }); + } + } + }); + + (join_handle, shutdown_tx) +} diff --git a/src/meta/src/rpc/server.rs b/src/meta/src/rpc/server.rs index 9d2e1384a6ea9..493a0738b7f1f 100644 --- a/src/meta/src/rpc/server.rs +++ b/src/meta/src/rpc/server.rs @@ -20,6 +20,7 @@ use either::Either; use etcd_client::ConnectOptions; use risingwave_backup::storage::ObjectStoreMetaSnapshotStorage; use risingwave_common::monitor::process_linux::monitor_process; +use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common_service::metrics_manager::MetricsManager; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::parse_remote_object_store; @@ -53,7 +54,7 @@ use crate::manager::{ SystemParamManager, }; use crate::rpc::election_client::{ElectionClient, EtcdElectionClient}; -use crate::rpc::metrics::MetaMetrics; +use crate::rpc::metrics::{start_worker_info_monitor, MetaMetrics}; use crate::rpc::service::backup_service::BackupServiceImpl; use crate::rpc::service::cluster_service::ClusterServiceImpl; use crate::rpc::service::heartbeat_service::HeartbeatServiceImpl; @@ -314,6 +315,10 @@ pub async fn start_service_as_election_leader( let registry = meta_metrics.registry(); monitor_process(registry).unwrap(); + let system_params_manager = + Arc::new(SystemParamManager::new(env.clone(), init_system_params).await?); + let system_params_reader: SystemParamsReader = system_params_manager.get_params().await.into(); + let cluster_manager = Arc::new( ClusterManager::new(env.clone(), max_heartbeat_interval) .await @@ -338,7 +343,7 @@ pub async fn start_service_as_election_leader( .await .unwrap(); - let meta_member_srv = MetaMemberServiceImpl::new(match election_client { + let meta_member_srv = MetaMemberServiceImpl::new(match election_client.clone() { None => Either::Right(address_info.clone()), Some(election_client) => Either::Left(election_client), }); @@ -360,8 +365,10 @@ pub async fn start_service_as_election_leader( tokio::spawn(dashboard_service.serve(address_info.ui_path)); } - let (barrier_scheduler, scheduled_barriers) = - BarrierScheduler::new_pair(hummock_manager.clone(), env.opts.checkpoint_frequency); + let (barrier_scheduler, scheduled_barriers) = BarrierScheduler::new_pair( + hummock_manager.clone(), + system_params_reader.checkpoint_frequency() as usize, + ); let source_manager = Arc::new( SourceManager::new( @@ -442,8 +449,6 @@ pub async fn start_service_as_election_leader( backup_manager.clone(), compactor_manager.clone(), )); - let system_params_manager = - Arc::new(SystemParamManager::new(env.clone(), init_system_params).await?); let ddl_srv = DdlServiceImpl::::new( env.clone(), @@ -490,7 +495,7 @@ pub async fn start_service_as_election_leader( ); let health_srv = HealthServiceImpl::new(); let backup_srv = BackupServiceImpl::new(backup_manager); - let system_params_srv = SystemParamsServiceImpl::new(system_params_manager); + let system_params_srv = SystemParamsServiceImpl::new(system_params_manager.clone()); if let Some(prometheus_addr) = address_info.prometheus_addr { MetricsManager::boot_metrics_service( @@ -509,8 +514,9 @@ pub async fn start_service_as_election_leader( let mut sub_tasks = hummock::start_hummock_workers(vacuum_manager, compaction_scheduler, &env.opts); sub_tasks.push( - ClusterManager::start_worker_num_monitor( + start_worker_info_monitor( cluster_manager.clone(), + election_client.clone(), Duration::from_secs(env.opts.node_num_monitor_interval_sec), meta_metrics.clone(), ) diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index ebcfcbaee8bc8..41315fb78060f 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -25,7 +25,9 @@ use risingwave_common::util::iter_util::ZipEqFast; 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::{Dispatcher, DispatcherType, MergeNode, StreamActor, StreamNode}; +use risingwave_pb::stream_plan::{ + DispatchStrategy, Dispatcher, DispatcherType, MergeNode, StreamActor, StreamNode, +}; use super::id::GlobalFragmentIdsExt; use super::Locations; @@ -339,14 +341,17 @@ impl ActorGraphBuildStateInner { /// Create a new hash dispatcher. fn new_hash_dispatcher( - column_indices: &[u32], + strategy: &DispatchStrategy, downstream_fragment_id: GlobalFragmentId, downstream_actors: &[GlobalActorId], downstream_actor_mapping: ActorMapping, ) -> Dispatcher { + assert_eq!(strategy.r#type(), DispatcherType::Hash); + Dispatcher { r#type: DispatcherType::Hash as _, - column_indices: column_indices.to_vec(), + dist_key_indices: strategy.dist_key_indices.clone(), + output_indices: strategy.output_indices.clone(), hash_mapping: Some(downstream_actor_mapping.to_protobuf()), dispatcher_id: downstream_fragment_id.as_global_id() as u64, downstream_actor_id: downstream_actors.as_global_ids(), @@ -355,14 +360,17 @@ impl ActorGraphBuildStateInner { /// Create a new dispatcher for non-hash types. fn new_normal_dispatcher( - dispatcher_type: DispatcherType, + strategy: &DispatchStrategy, downstream_fragment_id: GlobalFragmentId, downstream_actors: &[GlobalActorId], ) -> Dispatcher { - assert_ne!(dispatcher_type, DispatcherType::Hash); + assert_ne!(strategy.r#type(), DispatcherType::Hash); + assert!(strategy.dist_key_indices.is_empty()); + Dispatcher { - r#type: dispatcher_type as _, - column_indices: Vec::new(), + r#type: strategy.r#type, + dist_key_indices: vec![], + output_indices: strategy.output_indices.clone(), hash_mapping: None, dispatcher_id: downstream_fragment_id.as_global_id() as u64, downstream_actor_id: downstream_actors.as_global_ids(), @@ -441,7 +449,11 @@ impl ActorGraphBuildStateInner { // Create a new dispatcher just between these two actors. self.add_dispatcher( *upstream_id, - Self::new_normal_dispatcher(dt, downstream.fragment_id, &[*downstream_id]), + Self::new_normal_dispatcher( + &edge.dispatch_strategy, + downstream.fragment_id, + &[*downstream_id], + ), ); // Also record the upstream for the downstream actor. @@ -474,13 +486,17 @@ impl ActorGraphBuildStateInner { .to_actor(&downstream_locations); Self::new_hash_dispatcher( - &edge.dispatch_strategy.column_indices, + &edge.dispatch_strategy, downstream.fragment_id, downstream.actor_ids, actor_mapping, ) } else { - Self::new_normal_dispatcher(dt, downstream.fragment_id, downstream.actor_ids) + Self::new_normal_dispatcher( + &edge.dispatch_strategy, + downstream.fragment_id, + downstream.actor_ids, + ) }; for upstream_id in upstream.actor_ids { self.add_dispatcher(*upstream_id, dispatcher.clone()); diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 4820058f46c09..71cdd3db10cd9 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -132,6 +132,7 @@ impl BuildingFragment { } NodeBody::Dml(dml_node) => { dml_node.table_id = table_id; + dml_node.table_version_id = job.table_version_id().unwrap(); } _ => {} }); @@ -417,6 +418,13 @@ impl CompleteStreamFragmentGraph { .context("upstream materialized view fragment not found")?; let mview_id = GlobalFragmentId::new(mview_fragment.fragment_id); + // TODO: only output the fields that are used by the downstream `Chain`. + // https://github.com/risingwavelabs/risingwave/issues/4529 + let mview_output_indices = { + let nodes = mview_fragment.actors[0].nodes.as_ref().unwrap(); + (0..nodes.fields.len() as u32).collect() + }; + let edge = StreamFragmentEdge { id: EdgeId::UpstreamExternal { upstream_table_id, @@ -426,7 +434,8 @@ impl CompleteStreamFragmentGraph { // and the downstream `Chain` of the new materialized view. dispatch_strategy: DispatchStrategy { r#type: DispatcherType::NoShuffle as _, - ..Default::default() + dist_key_indices: vec![], // not used + output_indices: mview_output_indices, }, }; @@ -492,7 +501,8 @@ impl CompleteStreamFragmentGraph { // and the downstream `Chain` of the new materialized view. dispatch_strategy: DispatchStrategy { r#type: DispatcherType::NoShuffle as _, - ..Default::default() + dist_key_indices: vec![], // not used + output_indices: vec![], // FIXME: should erase the changes of the schema }, }; diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index 9dca3500cf8b9..a1c20fa8c8cbe 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -213,7 +213,8 @@ fn make_stream_fragments() -> Vec { node_body: Some(NodeBody::Exchange(ExchangeNode { strategy: Some(DispatchStrategy { r#type: DispatcherType::Hash as i32, - column_indices: vec![0], + dist_key_indices: vec![0], + output_indices: vec![0, 1, 2], }), })), fields: vec![ @@ -374,7 +375,8 @@ fn make_fragment_edges() -> Vec { StreamFragmentEdge { dispatch_strategy: Some(DispatchStrategy { r#type: DispatcherType::Simple as i32, - column_indices: vec![], + dist_key_indices: vec![], + output_indices: vec![], }), link_id: 4, upstream_id: 1, @@ -383,7 +385,8 @@ fn make_fragment_edges() -> Vec { StreamFragmentEdge { dispatch_strategy: Some(DispatchStrategy { r#type: DispatcherType::Hash as i32, - column_indices: vec![0], + dist_key_indices: vec![0], + output_indices: vec![], }), link_id: 1, upstream_id: 2, diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index ae646f7cb6467..cef14075d12b6 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -21,7 +21,7 @@ fail = "0.5" futures = { version = "0.3", default-features = false, features = ["alloc"] } hyper = "0.14" itertools = "0.10" -opendal = "0.26.2" +opendal = "0.27.2" prometheus = { version = "0.13", features = ["process"] } random-string = "1.0" risingwave_common = { path = "../common" } 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 eca30c3fe16c3..63c35b3b33f6d 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 @@ -150,9 +150,7 @@ impl ObjectStore for OpendalObjectStore { /// Deletes the objects with the given paths permanently from the storage. If an object /// specified in the request is not found, it will be considered as successfully deleted. async fn delete_objects(&self, paths: &[String]) -> ObjectResult<()> { - for path in paths { - self.op.object(path).delete().await?; - } + self.op.batch().remove(paths.to_vec()).await?; Ok(()) } @@ -287,29 +285,26 @@ mod tests { obj_store.delete(&path).await.unwrap(); } - // Currently OpenDAL does not support delete objects operation, but they are planning this - // feature. So let's not delete this unit test now. https://github.com/datafuselabs/opendal/issues/1279 - - // #[tokio::test] - // async fn test_memory_delete_objects() { - // let block1 = Bytes::from("123456"); - // let block2 = Bytes::from("987654"); - // let store = OpendalObjectStore::new_memory_engine().unwrap(); - // store.upload("/abc", block1).await.unwrap(); - // store.upload("/klm", block2).await.unwrap(); + #[tokio::test] + async fn test_memory_delete_objects() { + let block1 = Bytes::from("123456"); + let block2 = Bytes::from("987654"); + let store = OpendalObjectStore::new_memory_engine().unwrap(); + store.upload("abc", block1).await.unwrap(); + store.upload("/klm", block2).await.unwrap(); - // assert_eq!(store.list("").await.unwrap().len(), 2); + assert_eq!(store.list("").await.unwrap().len(), 2); - // let str_list = [ - // String::from("/abc"), - // String::from("/klm"), - // String::from("/xyz"), - // ]; + let str_list = [ + String::from("abc"), + String::from("klm"), + String::from("xyz"), + ]; - // store.delete_objects(&str_list).await.unwrap(); + store.delete_objects(&str_list).await.unwrap(); - // assert_eq!(store.list("").await.unwrap().len(), 0); - // } + assert_eq!(store.list("").await.unwrap().len(), 0); + } #[tokio::test] async fn test_memory_read_multi_block() { diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 9f11db934ccc3..278a149c70502 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -29,7 +29,6 @@ use futures::future::try_join_all; use futures::stream; use hyper::Body; use itertools::Itertools; -use random_string::generate; use tokio::io::AsyncRead; use tokio::task::JoinHandle; @@ -292,7 +291,6 @@ pub struct S3ObjectStore { part_size: usize, /// For S3 specific metrics. metrics: Arc, - object_store_use_batch_delete: bool, } #[async_trait::async_trait] @@ -433,12 +431,6 @@ impl ObjectStore for S3ObjectStore { async fn delete_objects(&self, paths: &[String]) -> ObjectResult<()> { // AWS restricts the number of objects per request to 1000. const MAX_LEN: usize = 1000; - if !self.object_store_use_batch_delete { - for path in paths { - self.delete(path).await?; - } - return Ok(()); - } // If needed, split given set into subsets of size with no more than `MAX_LEN` objects. for start_idx /* inclusive */ in (0..paths.len()).step_by(MAX_LEN) { @@ -531,7 +523,6 @@ impl S3ObjectStore { bucket, part_size: S3_PART_SIZE, metrics, - object_store_use_batch_delete: true, } } @@ -566,36 +557,11 @@ impl S3ObjectStore { let client = Client::new(&sdk_config); Self::configure_bucket_lifecycle(&client, bucket.as_str()).await; - // check whether use batch delete - let charset = "1234567890"; - let test_path = "risingwave_check_batch_delete/".to_string() + &generate(10, charset); - client - .put_object() - .bucket(&bucket) - .body(aws_sdk_s3::types::ByteStream::from(Bytes::from( - "test batch delete", - ))) - .key(&test_path) - .send() - .await - .unwrap(); - let obj_ids = vec![ObjectIdentifier::builder().key(&test_path).build()]; - - let delete_builder = Delete::builder().set_objects(Some(obj_ids)); - let object_store_use_batch_delete = client - .delete_objects() - .bucket(&bucket) - .delete(delete_builder.build()) - .send() - .await - .is_ok(); - Self { client, bucket: bucket.to_string(), part_size: S3_PART_SIZE, metrics, - object_store_use_batch_delete, } } @@ -630,7 +596,6 @@ impl S3ObjectStore { bucket: bucket.to_string(), part_size: MINIO_PART_SIZE, metrics, - object_store_use_batch_delete: true, } } diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index 265cde773c334..5e6821b60dd8f 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -54,7 +54,7 @@ mod stream_client; pub use compute_client::{ComputeClient, ComputeClientPool, ComputeClientPoolRef}; pub use connector_client::ConnectorClient; pub use hummock_meta_client::{CompactTaskItem, HummockMetaClient}; -pub use meta_client::{MetaClient, SystemParamsReader}; +pub use meta_client::MetaClient; pub use stream_client::{StreamClient, StreamClientPool, StreamClientPoolRef}; #[async_trait] diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index bf8b571146d50..e05e47fab5a97 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -25,7 +25,7 @@ use futures::stream::BoxStream; use itertools::Itertools; use risingwave_common::catalog::{CatalogVersion, FunctionId, IndexId, TableId}; use risingwave_common::config::MAX_CONNECTION_WINDOW_SIZE; -use risingwave_common::system_param::system_params_to_kv; +use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::util::addr::HostAddr; use risingwave_hummock_sdk::compact::CompactorRuntimeConfig; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; @@ -57,7 +57,7 @@ use risingwave_pb::meta::reschedule_request::Reschedule as ProstReschedule; use risingwave_pb::meta::scale_service_client::ScaleServiceClient; use risingwave_pb::meta::stream_manager_service_client::StreamManagerServiceClient; use risingwave_pb::meta::system_params_service_client::SystemParamsServiceClient; -use risingwave_pb::meta::{SystemParams as ProstSystemParams, *}; +use risingwave_pb::meta::*; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_pb::user::update_user_request::UpdateField; use risingwave_pb::user::user_service_client::UserServiceClient; @@ -70,7 +70,6 @@ use tokio::time; use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tonic::transport::{Channel, Endpoint}; use tonic::{Code, Streaming}; -use tracing::warn; use crate::error::{Result, RpcError}; use crate::hummock_meta_client::{CompactTaskItem, HummockMetaClient}; @@ -953,69 +952,6 @@ impl HummockMetaClient for MetaClient { } } -/// A wrapper for [`risingwave_pb::meta::SystemParams`] for 2 purposes: -/// - Avoid misuse of deprecated fields by hiding their getters. -/// - Abstract fallback logic for fields that might not be provided by meta service due to backward -/// compatibility. -pub struct SystemParamsReader { - prost: ProstSystemParams, -} - -impl From for SystemParamsReader { - fn from(prost: ProstSystemParams) -> Self { - Self { prost } - } -} - -impl SystemParamsReader { - pub fn barrier_interval_ms(&self) -> u32 { - self.prost.barrier_interval_ms.unwrap() - } - - pub fn checkpoint_frequency(&self) -> u64 { - self.prost.checkpoint_frequency.unwrap() - } - - pub fn sstable_size_mb(&self) -> u32 { - self.prost.sstable_size_mb.unwrap() - } - - pub fn block_size_kb(&self) -> u32 { - self.prost.block_size_kb.unwrap() - } - - pub fn bloom_false_positive(&self) -> f64 { - self.prost.bloom_false_positive.unwrap() - } - - // TODO(zhidong): Only read from system params in v0.1.18. - pub fn state_store(&self, from_local: String) -> String { - let from_prost = self.prost.state_store.as_ref().unwrap(); - if from_prost.is_empty() { - warn!("--state-store is not specified on meta node, reading from CLI instead"); - from_local - } else { - from_prost.clone() - } - } - - pub fn data_directory(&self) -> &str { - self.prost.data_directory.as_ref().unwrap() - } - - pub fn backup_storage_url(&self) -> &str { - self.prost.backup_storage_url.as_ref().unwrap() - } - - pub fn backup_storage_directory(&self) -> &str { - self.prost.backup_storage_directory.as_ref().unwrap() - } - - pub fn to_kv(&self) -> Vec<(String, String)> { - system_params_to_kv(&self.prost).unwrap() - } -} - #[derive(Debug, Clone)] struct GrpcMetaClientCore { cluster_client: ClusterServiceClient, diff --git a/src/source/benches/json_parser.rs b/src/source/benches/json_parser.rs index 92b286da7b1cb..22d4a1cb94e7f 100644 --- a/src/source/benches/json_parser.rs +++ b/src/source/benches/json_parser.rs @@ -130,7 +130,7 @@ fn get_descs() -> Vec { fn bench_json_parser(c: &mut Criterion) { let descs = get_descs(); - let parser = JsonParser::new(descs.clone()).unwrap(); + let parser = JsonParser::new_for_test(descs.clone()).unwrap(); let records = generate_all_json(); let rt = tokio::runtime::Builder::new_multi_thread() .enable_all() diff --git a/src/source/src/dml_manager.rs b/src/source/src/dml_manager.rs index de00117321328..21e19cff21c1e 100644 --- a/src/source/src/dml_manager.rs +++ b/src/source/src/dml_manager.rs @@ -92,9 +92,18 @@ impl DmlManager { // Register with the correct version. This happens when the following // `DmlExecutor`s of this table is activated on this compute // node. - Ordering::Equal => handle.upgrade().with_context(|| { - format!("fail to register reader for table with key `{table_id:?}`") - })?, + Ordering::Equal => handle + .upgrade() + .inspect(|handle| { + assert_eq!( + handle.column_descs(), + column_descs, + "dml handler registers with same version but different schema" + ) + }) + .with_context(|| { + format!("fail to register reader for table with key `{table_id:?}`") + })?, // A new version of the table is activated, overwrite the old reader. Ordering::Greater => new_handle!(o), @@ -254,4 +263,24 @@ mod tests { .write_chunk_ready(table_id, new_version_id, new_chunk()) .unwrap(); } + + #[test] + #[should_panic] + fn test_bad_schema() { + let dml_manager = DmlManager::new(); + let table_id = TableId::new(1); + let table_version_id = INITIAL_TABLE_VERSION_ID; + + let column_descs = vec![ColumnDesc::unnamed(100.into(), DataType::Float64)]; + let other_column_descs = vec![ColumnDesc::unnamed(101.into(), DataType::Float64)]; + + let _h = dml_manager + .register_reader(table_id, table_version_id, &column_descs) + .unwrap(); + + // Should panic as the schema is different. + let _h = dml_manager + .register_reader(table_id, table_version_id, &other_column_descs) + .unwrap(); + } } diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index fe6fb766cc832..07509b04f041a 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -100,6 +100,8 @@ impl SourceDescBuilder { ProstRowFormatType::CanalJson => SourceFormat::CanalJson, ProstRowFormatType::Native => SourceFormat::Native, ProstRowFormatType::DebeziumAvro => SourceFormat::DebeziumAvro, + ProstRowFormatType::UpsertJson => SourceFormat::UpsertJson, + ProstRowFormatType::UpsertAvro => SourceFormat::UpsertAvro, _ => unreachable!(), }; diff --git a/src/source/src/table.rs b/src/source/src/table.rs index e479868fc9de1..1e1b9e16f5d70 100644 --- a/src/source/src/table.rs +++ b/src/source/src/table.rs @@ -52,7 +52,6 @@ pub struct TableDmlHandle { core: RwLock, /// All columns in this table. - #[allow(dead_code)] column_descs: Vec, } @@ -123,6 +122,11 @@ impl TableDmlHandle { } } } + + /// Get the reference of all columns in this table. + pub(super) fn column_descs(&self) -> &[ColumnDesc] { + self.column_descs.as_ref() + } } #[easy_ext::ext(TableDmlHandleTestExt)] diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index 06ed47fa2cf38..41a01eec424eb 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -89,12 +89,14 @@ pub struct CreateSourceStatement { pub enum SourceSchema { Protobuf(ProtobufSchema), // Keyword::PROTOBUF ProtobufSchema - Json, // Keyword::JSON - DebeziumJson, // Keyword::DEBEZIUM_JSON - Avro(AvroSchema), // Keyword::AVRO - Maxwell, // Keyword::MAXWELL - CanalJson, // Keyword::CANAL_JSON - Csv(CsvInfo), // Keyword::CSV + Json, // Keyword::JSON + DebeziumJson, // Keyword::DEBEZIUM_JSON + UpsertJson, // Keyword::UPSERT_JSON + Avro(AvroSchema), // Keyword::AVRO + UpsertAvro(AvroSchema), // Keyword::UpsertAVRO + Maxwell, // Keyword::MAXWELL + CanalJson, // Keyword::CANAL_JSON + Csv(CsvInfo), // Keyword::CSV Native, DebeziumAvro(DebeziumAvroSchema), // Keyword::DEBEZIUM_AVRO } @@ -103,6 +105,8 @@ impl ParseTo for SourceSchema { fn parse_to(p: &mut Parser) -> Result { let schema = if p.parse_keywords(&[Keyword::JSON]) { SourceSchema::Json + } else if p.parse_keywords(&[Keyword::UPSERT_JSON]) { + SourceSchema::UpsertJson } else if p.parse_keywords(&[Keyword::PROTOBUF]) { impl_parse_to!(protobuf_schema: ProtobufSchema, p); SourceSchema::Protobuf(protobuf_schema) @@ -111,6 +115,9 @@ impl ParseTo for SourceSchema { } else if p.parse_keywords(&[Keyword::AVRO]) { impl_parse_to!(avro_schema: AvroSchema, p); SourceSchema::Avro(avro_schema) + } else if p.parse_keywords(&[Keyword::UPSERT_AVRO]) { + impl_parse_to!(avro_schema: AvroSchema, p); + SourceSchema::UpsertAvro(avro_schema) } else if p.parse_keywords(&[Keyword::MAXWELL]) { SourceSchema::Maxwell } else if p.parse_keywords(&[Keyword::CANAL_JSON]) { @@ -123,7 +130,7 @@ impl ParseTo for SourceSchema { SourceSchema::DebeziumAvro(avro_schema) } else { return Err(ParserError::ParserError( - "expected JSON | PROTOBUF | DEBEZIUM_JSON | DEBEZIUM_AVRO | AVRO | MAXWELL | CANAL_JSON after ROW FORMAT".to_string(), + "expected JSON | UPSERT_JSON | PROTOBUF | DEBEZIUM_JSON | DEBEZIUM_AVRO | AVRO | UPSERT_AVRO | MAXWELL | CANAL_JSON after ROW FORMAT".to_string(), )); }; Ok(schema) @@ -135,9 +142,11 @@ impl fmt::Display for SourceSchema { match self { SourceSchema::Protobuf(protobuf_schema) => write!(f, "PROTOBUF {}", protobuf_schema), SourceSchema::Json => write!(f, "JSON"), + SourceSchema::UpsertJson => write!(f, "UPSERT JSON"), SourceSchema::Maxwell => write!(f, "MAXWELL"), SourceSchema::DebeziumJson => write!(f, "DEBEZIUM JSON"), SourceSchema::Avro(avro_schema) => write!(f, "AVRO {}", avro_schema), + SourceSchema::UpsertAvro(avro_schema) => write!(f, "UPSERT AVRO {}", avro_schema), SourceSchema::CanalJson => write!(f, "CANAL JSON"), SourceSchema::Csv(csv_info) => write!(f, "CSV {}", csv_info), SourceSchema::Native => write!(f, "NATIVE"), @@ -188,28 +197,22 @@ impl fmt::Display for ProtobufSchema { } // sql_grammar!(AvroSchema { -// [Keyword::MESSAGE], -// message_name: AstString, -// [Keyword::ROW, Keyword::SCHEMA, Keyword::LOCATION], -// row_schema_location: AstString, +// [Keyword::ROW, Keyword::SCHEMA, Keyword::LOCATION, [Keyword::CONFLUENT, Keyword::SCHEMA, +// Keyword::REGISTRY]], row_schema_location: AstString, // }); #[derive(Debug, Clone, PartialEq, Eq, Hash)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] pub struct AvroSchema { - pub message_name: AstString, pub row_schema_location: AstString, pub use_schema_registry: bool, } impl ParseTo for AvroSchema { fn parse_to(p: &mut Parser) -> Result { - impl_parse_to!([Keyword::MESSAGE], p); - impl_parse_to!(message_name: AstString, p); impl_parse_to!([Keyword::ROW, Keyword::SCHEMA, Keyword::LOCATION], p); impl_parse_to!(use_schema_registry => [Keyword::CONFLUENT, Keyword::SCHEMA, Keyword::REGISTRY], p); impl_parse_to!(row_schema_location: AstString, p); Ok(Self { - message_name, row_schema_location, use_schema_registry, }) @@ -219,8 +222,6 @@ impl ParseTo for AvroSchema { impl fmt::Display for AvroSchema { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let mut v: Vec = vec![]; - impl_fmt_display!([Keyword::MESSAGE], v); - impl_fmt_display!(message_name, v, self); impl_fmt_display!([Keyword::ROW, Keyword::SCHEMA, Keyword::LOCATION], v); impl_fmt_display!(use_schema_registry => [Keyword::CONFLUENT, Keyword::SCHEMA, Keyword::REGISTRY], v, self); impl_fmt_display!(row_schema_location, v, self); diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index 3a60e5e9eed98..953fec2459df3 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -511,6 +511,8 @@ define_keywords!( UNNEST, UPDATE, UPPER, + UPSERT_AVRO, + UPSERT_JSON, USAGE, USER, USING, diff --git a/src/storage/backup/integration_tests/common.sh b/src/storage/backup/integration_tests/common.sh index d254e8716b015..672463df6a926 100644 --- a/src/storage/backup/integration_tests/common.sh +++ b/src/storage/backup/integration_tests/common.sh @@ -14,11 +14,11 @@ function clean_all_data { } function clean_etcd_data() { - cargo make clean-etcd-data + cargo make clean-etcd-data 1>/dev/null 2>&1 } function start_cluster() { - cargo make d ci-meta-backup-test + cargo make d ci-meta-backup-test 1>/dev/null 2>&1 } function wait_cluster_ready() { @@ -27,18 +27,18 @@ function wait_cluster_ready() { } function full_gc_sst() { - ${BACKUP_TEST_RW_ALL_IN_ONE} risectl hummock trigger-full-gc -s 0 + ${BACKUP_TEST_RW_ALL_IN_ONE} risectl hummock trigger-full-gc -s 0 1>/dev/null 2>&1 # TODO #6482: wait full gc finish deterministically. # Currently have to wait long enough. sleep 30 } function manual_compaction() { - ${BACKUP_TEST_RW_ALL_IN_ONE} risectl hummock trigger-manual-compaction "$@" + ${BACKUP_TEST_RW_ALL_IN_ONE} risectl hummock trigger-manual-compaction "$@" 1>/dev/null 2>&1 } function start_etcd_minio() { - cargo make d ci-meta-backup-test-restore + cargo make d ci-meta-backup-test-restore 1>/dev/null 2>&1 } function create_mvs() { @@ -78,14 +78,28 @@ function restore() { --meta-snapshot-id "${job_id}" \ --etcd-endpoints 127.0.0.1:2388 \ --storage-directory backup \ - --storage-url minio://hummockadmin:hummockadmin@127.0.0.1:9301/hummock001 + --storage-url minio://hummockadmin:hummockadmin@127.0.0.1:9301/hummock001 \ + 1>/dev/null } function execute_sql() { local sql sql=$1 - echo "execute sql ${sql}" - echo "SET QUERY_MODE=distributed;${sql}" | psql -h localhost -p 4566 -d dev -U root 2>&1 + echo "${sql}" | psql -h localhost -p 4566 -d dev -U root 2>&1 +} + +function execute_sql_and_expect() { + local sql + sql=$1 + local expected + expected=$2 + + echo "execute SQL ${sql}" + echo "expected string in result: ${expected}" + query_result=$(execute_sql "${sql}") + printf "actual result:\n%s\n" "${query_result}" + result=$(echo "${query_result}" | grep "${expected}") + [ -n "${result}" ] } function get_max_committed_epoch() { diff --git a/src/storage/backup/integration_tests/run_all.sh b/src/storage/backup/integration_tests/run_all.sh index 139b7bc0e1abe..b89e2a77a0040 100644 --- a/src/storage/backup/integration_tests/run_all.sh +++ b/src/storage/backup/integration_tests/run_all.sh @@ -11,3 +11,5 @@ for t in "${tests[@]}" do bash "${DIR}/${t}" done + +echo "all tests succeeded" \ No newline at end of file diff --git a/src/storage/backup/integration_tests/test_query_backup.sh b/src/storage/backup/integration_tests/test_query_backup.sh index b2c901c720fa2..40a762f661734 100644 --- a/src/storage/backup/integration_tests/test_query_backup.sh +++ b/src/storage/backup/integration_tests/test_query_backup.sh @@ -69,48 +69,35 @@ done echo "safe epoch after compaction: ${safe_epoch}" echo "QUERY_EPOCH=safe_epoch. It should fail because it's not covered by any backup" -result=$( -execute_sql " -SET QUERY_EPOCH TO ${safe_epoch}; -select * from t1; -" | grep "Read backup error backup include epoch ${safe_epoch} not found" -) -[ -n "${result}" ] +execute_sql_and_expect \ +"SET QUERY_EPOCH TO ${safe_epoch}; +select * from t1;" \ +"Read backup error backup include epoch ${safe_epoch} not found" echo "QUERY_EPOCH=0 aka disabling query backup" -result=$( -execute_sql " -SET QUERY_EPOCH TO 0; -select * from t1; -" | grep "1 row" -) -[ -n "${result}" ] +execute_sql_and_expect \ +"SET QUERY_EPOCH TO 0; +select * from t1;" \ +"1 row" echo "QUERY_EPOCH=backup_safe_epoch + 1, it's < safe_epoch but covered by backup" [ $((backup_safe_epoch + 1)) -eq 1 ] -result=$( -execute_sql " -SET QUERY_EPOCH TO $((backup_safe_epoch + 1)); -select * from t1; -" | grep "0 row" -) -[ -n "${result}" ] +execute_sql_and_expect \ +"SET QUERY_EPOCH TO $((backup_safe_epoch + 1)); +select * from t1;" \ +"0 row" echo "QUERY_EPOCH=backup_mce < safe_epoch, it's < safe_epoch but covered by backup" -result=$( -execute_sql " -SET QUERY_EPOCH TO $((backup_mce)); -select * from t1; -" | grep "3 row" -) -[ -n "${result}" ] +execute_sql_and_expect \ +"SET QUERY_EPOCH TO ${backup_mce}; +select * from t1;" \ +"3 row" echo "QUERY_EPOCH=future epoch. It should fail because it's not covered by any backup" future_epoch=18446744073709551615 -result=$( -execute_sql " -SET QUERY_EPOCH TO ${future_epoch}; -select * from t1; -" | grep "Read backup error backup include epoch ${future_epoch} not found" -) -[ -n "${result}" ] \ No newline at end of file +execute_sql_and_expect \ +"SET QUERY_EPOCH TO ${future_epoch}; +select * from t1;" \ +"Read backup error backup include epoch ${future_epoch} not found" + +echo "test succeeded" \ No newline at end of file diff --git a/src/storage/hummock_test/src/hummock_read_version_tests.rs b/src/storage/hummock_test/src/hummock_read_version_tests.rs index b59b9a89daa5a..ad441f0505b8c 100644 --- a/src/storage/hummock_test/src/hummock_read_version_tests.rs +++ b/src/storage/hummock_test/src/hummock_read_version_tests.rs @@ -158,6 +158,8 @@ async fn test_read_version_basic() { stale_key_count: 1, total_key_count: 1, divide_version: 0, + min_epoch: 0, + max_epoch: 0, }), LocalSstableInfo::for_test(SstableInfo { id: 2, @@ -172,6 +174,8 @@ async fn test_read_version_basic() { stale_key_count: 1, total_key_count: 1, divide_version: 0, + min_epoch: 0, + max_epoch: 0, }), ], epoch_id_vec_for_clear, diff --git a/src/storage/hummock_test/src/lib.rs b/src/storage/hummock_test/src/lib.rs index 41d57a3bb6571..12bbe52ee8743 100644 --- a/src/storage/hummock_test/src/lib.rs +++ b/src/storage/hummock_test/src/lib.rs @@ -23,8 +23,6 @@ mod compactor_tests; #[cfg(all(test, feature = "failpoints"))] mod failpoint_tests; #[cfg(test)] -mod local_version_manager_tests; -#[cfg(test)] mod snapshot_tests; #[cfg(test)] mod state_store_tests; diff --git a/src/storage/hummock_test/src/local_version_manager_tests.rs b/src/storage/hummock_test/src/local_version_manager_tests.rs index 75e7acc99ff4b..6e2f507785a9e 100644 --- a/src/storage/hummock_test/src/local_version_manager_tests.rs +++ b/src/storage/hummock_test/src/local_version_manager_tests.rs @@ -32,9 +32,6 @@ use risingwave_pb::hummock::HummockVersion; use risingwave_storage::hummock::compactor::CompactorContext; use risingwave_storage::hummock::event_handler::hummock_event_handler::BufferTracker; use risingwave_storage::hummock::iterator::test_utils::mock_sstable_store; -use risingwave_storage::hummock::local_version::local_version_manager::{ - LocalVersionManager, LocalVersionManagerRef, -}; use risingwave_storage::hummock::shared_buffer::shared_buffer_batch::SharedBufferBatch; use risingwave_storage::hummock::shared_buffer::UncommittedData; use risingwave_storage::hummock::test_utils::{ diff --git a/src/storage/hummock_test/src/snapshot_tests.rs b/src/storage/hummock_test/src/snapshot_tests.rs index 81c0e9e30604c..998c9934d1316 100644 --- a/src/storage/hummock_test/src/snapshot_tests.rs +++ b/src/storage/hummock_test/src/snapshot_tests.rs @@ -18,21 +18,12 @@ use std::sync::Arc; use bytes::Bytes; use futures::TryStreamExt; use risingwave_hummock_sdk::HummockReadEpoch; -use risingwave_meta::hummock::test_utils::setup_compute_env; use risingwave_meta::hummock::MockHummockMetaClient; use risingwave_rpc_client::HummockMetaClient; -use risingwave_storage::hummock::iterator::test_utils::mock_sstable_store; -use risingwave_storage::hummock::test_utils::default_opts_for_test; -use risingwave_storage::hummock::*; -use risingwave_storage::monitor::{CompactorMetrics, HummockStateStoreMetrics}; use risingwave_storage::storage_value::StorageValue; -use risingwave_storage::store::{ReadOptions, StateStoreWrite, WriteOptions}; -use risingwave_storage::StateStore; +use risingwave_storage::store::{ReadOptions, WriteOptions}; -use crate::get_notification_client_for_test; -use crate::test_utils::{ - with_hummock_storage_v1, with_hummock_storage_v2, HummockStateStoreTestTrait, -}; +use crate::test_utils::{with_hummock_storage_v2, HummockStateStoreTestTrait}; macro_rules! assert_count_range_scan { ($storage:expr, $range:expr, $expect_count:expr, $epoch:expr) => {{ @@ -271,204 +262,38 @@ async fn test_snapshot_range_scan_inner( assert_count_range_scan!(hummock_storage, .., 4, epoch); } -#[ignore] -async fn test_snapshot_backward_range_scan_inner(enable_sync: bool, enable_commit: bool) { - let sstable_store = mock_sstable_store(); - let hummock_options = Arc::new(default_opts_for_test()); - let (env, hummock_manager_ref, _cluster_manager_ref, worker_node) = - setup_compute_env(8080).await; - let mock_hummock_meta_client = Arc::new(MockHummockMetaClient::new( - hummock_manager_ref.clone(), - worker_node.id, - )); - - // TODO: may also test for v2 when the unit test is enabled. - let hummock_storage = HummockStorageV1::new( - hummock_options, - sstable_store, - mock_hummock_meta_client.clone(), - get_notification_client_for_test(env, hummock_manager_ref, worker_node), - Arc::new(HummockStateStoreMetrics::unused()), - Arc::new(risingwave_tracing::RwTracingService::disabled()), - Arc::new(CompactorMetrics::unused()), - ) - .await - .unwrap(); - - let epoch = 1; - hummock_storage - .ingest_batch( - vec![ - (Bytes::from("1"), StorageValue::new_put("test")), - (Bytes::from("2"), StorageValue::new_put("test")), - (Bytes::from("3"), StorageValue::new_put("test")), - (Bytes::from("4"), StorageValue::new_put("test")), - (Bytes::from("5"), StorageValue::new_put("test")), - (Bytes::from("6"), StorageValue::new_put("test")), - ], - vec![], - WriteOptions { - epoch, - table_id: Default::default(), - }, - ) - .await - .unwrap(); - if enable_sync { - let ssts = hummock_storage - .seal_and_sync_epoch(epoch) - .await - .unwrap() - .uncommitted_ssts; - if enable_commit { - mock_hummock_meta_client - .commit_epoch(epoch, ssts) - .await - .unwrap(); - hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch)) - .await - .unwrap(); - } - } - hummock_storage - .ingest_batch( - vec![ - (Bytes::from("5"), StorageValue::new_put("test")), - (Bytes::from("6"), StorageValue::new_put("test")), - (Bytes::from("7"), StorageValue::new_put("test")), - (Bytes::from("8"), StorageValue::new_put("test")), - ], - vec![], - WriteOptions { - epoch: epoch + 1, - table_id: Default::default(), - }, - ) - .await - .unwrap(); - if enable_sync { - let ssts = hummock_storage - .seal_and_sync_epoch(epoch + 1) - .await - .unwrap() - .uncommitted_ssts; - if enable_commit { - mock_hummock_meta_client - .commit_epoch(epoch + 1, ssts) - .await - .unwrap(); - hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch + 1)) - .await - .unwrap(); - } - } - - #[allow(unused_macros)] - macro_rules! key { - ($idx:expr) => { - Bytes::from(stringify!($idx)) - }; - } - - // TODO: re-enable it when backward range scan is supported again on hummock - // assert_count_backward_range_scan!(hummock_storage, key!(3)..=key!(2), 2, epoch); - // assert_count_backward_range_scan!(hummock_storage, key!(3)..key!(2), 1, epoch); - // assert_count_backward_range_scan!(hummock_storage, key!(3)..key!(1), 2, epoch); - // assert_count_backward_range_scan!(hummock_storage, key!(3)..=key!(1), 3, epoch); - // assert_count_backward_range_scan!(hummock_storage, key!(3)..key!(0), 3, epoch); - // assert_count_backward_range_scan!(hummock_storage, .., 6, epoch); - // assert_count_backward_range_scan!(hummock_storage, .., 8, epoch + 1); - // assert_count_backward_range_scan!(hummock_storage, key!(7)..key!(2), 5, epoch + 1); -} - -#[tokio::test] -async fn test_snapshot_v1() { - let (storage, meta_client) = with_hummock_storage_v1().await; - test_snapshot_inner(storage, meta_client, false, false).await; -} - #[tokio::test] async fn test_snapshot_v2() { let (storage, meta_client) = with_hummock_storage_v2(Default::default()).await; test_snapshot_inner(storage, meta_client, false, false).await; } -#[tokio::test] -async fn test_snapshot_with_sync_v1() { - let (storage, meta_client) = with_hummock_storage_v1().await; - test_snapshot_inner(storage, meta_client, true, false).await; -} - #[tokio::test] async fn test_snapshot_with_sync_v2() { let (storage, meta_client) = with_hummock_storage_v2(Default::default()).await; test_snapshot_inner(storage, meta_client, true, false).await; } -#[tokio::test] -async fn test_snapshot_with_commit_v1() { - let (storage, meta_client) = with_hummock_storage_v1().await; - test_snapshot_inner(storage, meta_client, true, true).await; -} - #[tokio::test] async fn test_snapshot_with_commit_v2() { let (storage, meta_client) = with_hummock_storage_v2(Default::default()).await; test_snapshot_inner(storage, meta_client, true, true).await; } -#[tokio::test] -async fn test_snapshot_range_scan_v1() { - let (storage, meta_client) = with_hummock_storage_v1().await; - test_snapshot_range_scan_inner(storage, meta_client, false, false).await; -} - #[tokio::test] async fn test_snapshot_range_scan_v2() { let (storage, meta_client) = with_hummock_storage_v2(Default::default()).await; test_snapshot_range_scan_inner(storage, meta_client, false, false).await; } -#[tokio::test] -async fn test_snapshot_range_scan_with_sync_v1() { - let (storage, meta_client) = with_hummock_storage_v1().await; - test_snapshot_range_scan_inner(storage, meta_client, true, false).await; -} - #[tokio::test] async fn test_snapshot_range_scan_with_sync_v2() { let (storage, meta_client) = with_hummock_storage_v2(Default::default()).await; test_snapshot_range_scan_inner(storage, meta_client, true, false).await; } -#[tokio::test] -async fn test_snapshot_range_scan_with_commit_v1() { - let (storage, meta_client) = with_hummock_storage_v1().await; - test_snapshot_range_scan_inner(storage, meta_client, true, true).await; -} - #[tokio::test] async fn test_snapshot_range_scan_with_commit_v2() { let (storage, meta_client) = with_hummock_storage_v2(Default::default()).await; test_snapshot_range_scan_inner(storage, meta_client, true, true).await; } - -#[ignore] -#[tokio::test] -async fn test_snapshot_backward_range_scan() { - test_snapshot_backward_range_scan_inner(false, false).await; -} - -#[ignore] -#[tokio::test] -async fn test_snapshot_backward_range_scan_with_sync() { - test_snapshot_backward_range_scan_inner(true, false).await; -} - -#[ignore] -#[tokio::test] -async fn test_snapshot_backward_range_scan_with_commit() { - test_snapshot_backward_range_scan_inner(true, true).await; -} diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index 168225312a7ae..e1329e5851bf6 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -26,8 +26,7 @@ use risingwave_meta::hummock::MockHummockMetaClient; use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::hummock::iterator::test_utils::mock_sstable_store; use risingwave_storage::hummock::test_utils::{count_stream, default_opts_for_test}; -use risingwave_storage::hummock::{HummockStorage, HummockStorageV1}; -use risingwave_storage::monitor::{CompactorMetrics, HummockStateStoreMetrics}; +use risingwave_storage::hummock::HummockStorage; use risingwave_storage::storage_value::StorageValue; use risingwave_storage::store::{ ReadOptions, StateStore, StateStoreRead, StateStoreWrite, SyncResult, WriteOptions, @@ -35,8 +34,7 @@ use risingwave_storage::store::{ use crate::get_notification_client_for_test; use crate::test_utils::{ - with_hummock_storage_v1, with_hummock_storage_v2, HummockStateStoreTestTrait, - HummockV2MixedStateStore, + with_hummock_storage_v2, HummockStateStoreTestTrait, HummockV2MixedStateStore, }; #[tokio::test] @@ -75,12 +73,6 @@ async fn test_empty_read_v2() { assert!(stream.try_next().await.unwrap().is_none()); } -#[tokio::test] -async fn test_basic_v1() { - let (hummock_storage, meta_client) = with_hummock_storage_v1().await; - test_basic_inner(hummock_storage, meta_client).await; -} - #[tokio::test] async fn test_basic_v2() { let (hummock_storage, meta_client) = with_hummock_storage_v2(Default::default()).await; @@ -427,12 +419,6 @@ async fn test_basic_inner( assert!(value.is_none()); } -#[tokio::test] -async fn test_state_store_sync_v1() { - let (hummock_storage, meta_client) = with_hummock_storage_v1().await; - test_state_store_sync_inner(hummock_storage, meta_client).await; -} - #[tokio::test] async fn test_state_store_sync_v2() { let (hummock_storage, meta_client) = with_hummock_storage_v2(Default::default()).await; @@ -538,27 +524,7 @@ async fn test_reload_storage() { let hummock_options = Arc::new(default_opts_for_test()); let (env, hummock_manager_ref, _cluster_manager_ref, worker_node) = setup_compute_env(8080).await; - let meta_client = Arc::new(MockHummockMetaClient::new( - hummock_manager_ref.clone(), - worker_node.id, - )); - - // TODO: may also test for v2 when the unit test is enabled. - let hummock_storage = HummockStorageV1::new( - hummock_options.clone(), - sstable_store.clone(), - meta_client.clone(), - get_notification_client_for_test( - env.clone(), - hummock_manager_ref.clone(), - worker_node.clone(), - ), - Arc::new(HummockStateStoreMetrics::unused()), - Arc::new(risingwave_tracing::RwTracingService::disabled()), - Arc::new(CompactorMetrics::unused()), - ) - .await - .unwrap(); + let (hummock_storage, meta_client) = with_hummock_storage_v2(Default::default()).await; let anchor = Bytes::from("aa"); // First batch inserts the anchor and others. @@ -754,12 +720,6 @@ async fn test_reload_storage() { assert_eq!(len, 3); } -#[tokio::test] -async fn test_write_anytime_v1() { - let (hummock_storage, meta_client) = with_hummock_storage_v1().await; - test_write_anytime_inner(hummock_storage, meta_client).await; -} - #[tokio::test] async fn test_write_anytime_v2() { let (hummock_storage, meta_client) = with_hummock_storage_v2(Default::default()).await; @@ -1054,12 +1014,6 @@ async fn test_write_anytime_inner( assert!(!ssts2.is_empty()); } -#[tokio::test] -async fn test_delete_get_v1() { - let (hummock_storage, meta_client) = with_hummock_storage_v1().await; - test_delete_get_inner(hummock_storage, meta_client).await; -} - #[tokio::test] async fn test_delete_get_v2() { let (hummock_storage, meta_client) = with_hummock_storage_v2(Default::default()).await; @@ -1134,12 +1088,6 @@ async fn test_delete_get_inner( .is_none()); } -#[tokio::test] -async fn test_multiple_epoch_sync_v1() { - let (hummock_storage, meta_client) = with_hummock_storage_v1().await; - test_multiple_epoch_sync_inner(hummock_storage, meta_client).await; -} - #[tokio::test] async fn test_multiple_epoch_sync_v2() { let (hummock_storage, meta_client) = with_hummock_storage_v2(Default::default()).await; diff --git a/src/storage/hummock_test/src/test_utils.rs b/src/storage/hummock_test/src/test_utils.rs index 2043a8802ece8..4fbb576e4685c 100644 --- a/src/storage/hummock_test/src/test_utils.rs +++ b/src/storage/hummock_test/src/test_utils.rs @@ -48,8 +48,7 @@ use risingwave_storage::hummock::local_version::pinned_version::PinnedVersion; use risingwave_storage::hummock::observer_manager::HummockObserverNode; use risingwave_storage::hummock::store::state_store::LocalHummockStorage; use risingwave_storage::hummock::test_utils::default_opts_for_test; -use risingwave_storage::hummock::{HummockStorage, HummockStorageV1}; -use risingwave_storage::monitor::{CompactorMetrics, HummockStateStoreMetrics}; +use risingwave_storage::hummock::HummockStorage; use risingwave_storage::storage_value::StorageValue; use risingwave_storage::store::*; use risingwave_storage::{ @@ -269,44 +268,6 @@ impl HummockStateStoreTestTrait for HummockV2MixedStateStore { } } -impl HummockStateStoreTestTrait for HummockStorageV1 { - fn get_pinned_version(&self) -> PinnedVersion { - self.get_pinned_version() - } -} - -pub async fn with_hummock_storage_v1() -> (HummockStorageV1, Arc) { - let sstable_store = mock_sstable_store(); - let hummock_options = Arc::new(default_opts_for_test()); - let (env, hummock_manager_ref, _cluster_manager_ref, worker_node) = - setup_compute_env(8080).await; - let meta_client = Arc::new(MockHummockMetaClient::new( - hummock_manager_ref.clone(), - worker_node.id, - )); - - let hummock_storage = HummockStorageV1::new( - hummock_options, - sstable_store, - meta_client.clone(), - get_notification_client_for_test(env, hummock_manager_ref.clone(), worker_node), - Arc::new(HummockStateStoreMetrics::unused()), - Arc::new(risingwave_tracing::RwTracingService::disabled()), - Arc::new(CompactorMetrics::unused()), - ) - .await - .unwrap(); - - register_tables_with_id_for_test( - hummock_storage.filter_key_extractor_manager(), - &hummock_manager_ref, - &[0], - ) - .await; - - (hummock_storage, meta_client) -} - pub async fn with_hummock_storage_v2( table_id: TableId, ) -> (HummockV2MixedStateStore, Arc) { diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index 31a8fbf145946..4459070df1b8f 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -826,6 +826,8 @@ mod tests { stale_key_count: 0, total_key_count: 0, divide_version: 0, + min_epoch: 0, + max_epoch: 0, })] } diff --git a/src/storage/src/hummock/local_version/local_version_manager.rs b/src/storage/src/hummock/local_version/local_version_manager.rs deleted file mode 100644 index 07b6b240e6330..0000000000000 --- a/src/storage/src/hummock/local_version/local_version_manager.rs +++ /dev/null @@ -1,384 +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 std::collections::HashMap; -use std::ops::RangeBounds; -use std::sync::Arc; - -use bytes::Bytes; -use parking_lot::{RwLock, RwLockWriteGuard}; -use risingwave_common::catalog::TableId; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionUpdateExt; -use risingwave_hummock_sdk::key::TableKey; -use risingwave_hummock_sdk::CompactionGroupId; -use risingwave_pb::hummock::version_update_payload; -use risingwave_pb::hummock::version_update_payload::Payload; -use tokio::task::JoinHandle; -use tracing::{error, info}; - -use crate::hummock::compactor::CompactorContext; -use crate::hummock::event_handler::hummock_event_handler::BufferTracker; -use crate::hummock::local_version::pinned_version::PinnedVersion; -use crate::hummock::local_version::{LocalVersion, ReadVersion, SyncUncommittedDataStage}; -use crate::hummock::shared_buffer::shared_buffer_batch::SharedBufferBatch; -use crate::hummock::shared_buffer::shared_buffer_uploader::{ - SharedBufferUploader, UploadTaskPayload, -}; -use crate::hummock::shared_buffer::OrderIndex; -use crate::hummock::utils::validate_table_key_range; -use crate::hummock::{HummockEpoch, HummockError, HummockResult, SstableIdManagerRef, TrackerId}; -use crate::storage_value::StorageValue; -use crate::store::SyncResult; - -pub type LocalVersionManagerRef = Arc; - -/// The `LocalVersionManager` maintains a local copy of storage service's hummock version data. -/// By acquiring a `ScopedLocalVersion`, the `Sstables` of this version is guaranteed to be valid -/// during the lifetime of `ScopedLocalVersion`. Internally `LocalVersionManager` will pin/unpin the -/// versions in storage service. -pub struct LocalVersionManager { - pub(crate) local_version: RwLock, - buffer_tracker: BufferTracker, - shared_buffer_uploader: Arc, - sstable_id_manager: SstableIdManagerRef, -} - -impl LocalVersionManager { - pub fn new( - pinned_version: PinnedVersion, - compactor_context: Arc, - buffer_tracker: BufferTracker, - ) -> Arc { - assert!(pinned_version.is_valid()); - let sstable_id_manager = compactor_context.sstable_id_manager.clone(); - - Arc::new(LocalVersionManager { - local_version: RwLock::new(LocalVersion::new(pinned_version)), - buffer_tracker, - shared_buffer_uploader: Arc::new(SharedBufferUploader::new(compactor_context)), - sstable_id_manager, - }) - } - - pub fn get_buffer_tracker(&self) -> &BufferTracker { - &self.buffer_tracker - } - - /// Updates cached version if the new version is of greater id. - /// You shouldn't unpin even the method returns false, as it is possible `hummock_version` is - /// being referenced by some readers. - pub fn try_update_pinned_version( - &self, - pin_resp_payload: version_update_payload::Payload, - ) -> Option { - let old_version = self.local_version.read(); - let new_version_id = match &pin_resp_payload { - Payload::VersionDeltas(version_deltas) => match version_deltas.version_deltas.last() { - Some(version_delta) => version_delta.id, - None => old_version.pinned_version().id(), - }, - Payload::PinnedVersion(version) => version.get_id(), - }; - - if old_version.pinned_version().id() >= new_version_id { - return None; - } - - let newly_pinned_version = match pin_resp_payload { - Payload::VersionDeltas(mut version_deltas) => { - old_version.filter_local_sst(&mut version_deltas); - let mut version_to_apply = old_version.pinned_version().version(); - for version_delta in &version_deltas.version_deltas { - assert_eq!(version_to_apply.id, version_delta.prev_id); - version_to_apply.apply_version_delta(version_delta); - } - version_to_apply - } - Payload::PinnedVersion(version) => version, - }; - - validate_table_key_range(&newly_pinned_version); - - drop(old_version); - let mut new_version = self.local_version.write(); - // check again to prevent other thread changes new_version. - if new_version.pinned_version().id() >= newly_pinned_version.get_id() { - return None; - } - - self.sstable_id_manager - .remove_watermark_sst_id(TrackerId::Epoch(newly_pinned_version.max_committed_epoch)); - new_version.set_pinned_version(newly_pinned_version); - let result = new_version.pinned_version().clone(); - RwLockWriteGuard::unlock_fair(new_version); - - Some(result) - } - - pub fn write_shared_buffer_batch(&self, batch: SharedBufferBatch) { - self.write_shared_buffer_inner(batch.epoch(), batch); - } - - pub async fn write_shared_buffer( - &self, - epoch: HummockEpoch, - kv_pairs: Vec<(Bytes, StorageValue)>, - delete_ranges: Vec<(Bytes, Bytes)>, - table_id: TableId, - ) -> HummockResult { - let sorted_items = SharedBufferBatch::build_shared_buffer_item_batches(kv_pairs); - let size = SharedBufferBatch::measure_batch_size(&sorted_items); - let batch = SharedBufferBatch::build_shared_buffer_batch( - epoch, - sorted_items, - size, - delete_ranges, - table_id, - Some( - self.buffer_tracker - .get_memory_limiter() - .require_memory(size as u64) - .await, - ), - ); - let batch_size = batch.size(); - self.write_shared_buffer_inner(batch.epoch(), batch); - Ok(batch_size) - } - - pub(crate) fn write_shared_buffer_inner(&self, epoch: HummockEpoch, batch: SharedBufferBatch) { - let mut local_version_guard = self.local_version.write(); - let sealed_epoch = local_version_guard.get_sealed_epoch(); - assert!( - epoch > sealed_epoch, - "write epoch must greater than max current epoch, write epoch {}, sealed epoch {}", - epoch, - sealed_epoch - ); - // Write into shared buffer - - let shared_buffer = match local_version_guard.get_mut_shared_buffer(epoch) { - Some(shared_buffer) => shared_buffer, - None => local_version_guard - .new_shared_buffer(epoch, self.buffer_tracker.global_upload_task_size().clone()), - }; - // The batch will be synced to S3 asynchronously if it is a local batch - shared_buffer.write_batch(batch); - } - - /// Issue a concurrent upload task to flush some local shared buffer batch to object store. - /// - /// This method should only be called in the buffer tracker worker. - /// - /// Return: - /// - Some(task join handle) when there is new upload task - /// - None when there is no new task - #[expect(dead_code)] - pub(in crate::hummock) fn flush_shared_buffer( - self: Arc, - ) -> Option<(HummockEpoch, JoinHandle<()>)> { - let (epoch, (order_index, payload, task_write_batch_size), compaction_group_index) = { - let mut local_version_guard = self.local_version.write(); - - // The current implementation is a trivial one, which issue only one flush task and wait - // for the task to finish. - let mut task = None; - let compaction_group_index = - local_version_guard.pinned_version.compaction_group_index(); - for (epoch, shared_buffer) in local_version_guard.iter_mut_unsynced_shared_buffer() { - if let Some(upload_task) = shared_buffer.new_upload_task() { - task = Some((*epoch, upload_task, compaction_group_index)); - break; - } - } - match task { - Some(task) => task, - None => return None, - } - }; - - let join_handle = tokio::spawn(async move { - info!( - "running flush task in epoch {} of size {}", - epoch, task_write_batch_size - ); - // TODO: may apply different `is_local` according to whether local spill is enabled. - let _ = self - .run_flush_upload_task(order_index, epoch, payload, compaction_group_index) - .await - .inspect_err(|err| { - error!( - "upload task fail. epoch: {}, order_index: {}. Err: {:?}", - epoch, order_index, err - ); - }); - info!( - "flush task in epoch {} of size {} finished", - epoch, task_write_batch_size - ); - }); - Some((epoch, join_handle)) - } - - #[cfg(any(test, feature = "test"))] - pub async fn sync_shared_buffer(&self, epoch: HummockEpoch) -> HummockResult { - self.seal_epoch(epoch, true); - self.await_sync_shared_buffer(epoch).await - } - - /// send event to `event_handler` thaen seal epoch in local version. - pub fn seal_epoch(&self, epoch: HummockEpoch, is_checkpoint: bool) { - self.local_version.write().seal_epoch(epoch, is_checkpoint); - } - - pub async fn await_sync_shared_buffer(&self, epoch: HummockEpoch) -> HummockResult { - tracing::trace!("sync epoch {}", epoch); - - let future = { - // Wait all epochs' task that less than epoch. - let mut local_version_guard = self.local_version.write(); - let (payload, sync_size) = local_version_guard.start_syncing(epoch); - let compaction_group_index = local_version_guard - .pinned_version() - .compaction_group_index(); - self.run_sync_upload_task(payload, compaction_group_index, sync_size, epoch) - }; - future.await?; - let local_version_guard = self.local_version.read(); - let sync_data = local_version_guard - .sync_uncommitted_data - .get(&epoch) - .expect("should exist"); - match sync_data.stage() { - SyncUncommittedDataStage::CheckpointEpochSealed(_) - | SyncUncommittedDataStage::Syncing(_) => { - unreachable!("when a join handle is finished, the stage should be synced or failed") - } - SyncUncommittedDataStage::Failed(_) => Err(HummockError::other("sync task failed")), - SyncUncommittedDataStage::Synced(ssts, sync_size) => { - let ssts = ssts.clone(); - let sync_size = *sync_size; - drop(local_version_guard); - Ok(SyncResult { - sync_size, - uncommitted_ssts: ssts, - }) - } - } - } - - pub async fn run_sync_upload_task( - &self, - task_payload: UploadTaskPayload, - compaction_group_index: Arc>, - sync_size: usize, - epoch: HummockEpoch, - ) -> HummockResult<()> { - match self - .shared_buffer_uploader - .flush(task_payload, epoch, compaction_group_index) - .await - { - Ok(ssts) => { - self.local_version - .write() - .data_synced(epoch, ssts, sync_size); - Ok(()) - } - Err(e) => { - self.local_version.write().fail_epoch_sync(epoch); - Err(e) - } - } - } - - #[expect(dead_code)] - async fn run_flush_upload_task( - &self, - order_index: OrderIndex, - epoch: HummockEpoch, - task_payload: UploadTaskPayload, - compaction_group_index: Arc>, - ) -> HummockResult<()> { - let task_result = self - .shared_buffer_uploader - .flush(task_payload, epoch, compaction_group_index) - .await; - - let mut local_version_guard = self.local_version.write(); - let shared_buffer_guard = local_version_guard - .get_mut_shared_buffer(epoch) - .expect("shared buffer should exist since some uncommitted data is not committed yet"); - - match task_result { - Ok(ssts) => { - shared_buffer_guard.succeed_upload_task(order_index, ssts); - Ok(()) - } - Err(e) => { - shared_buffer_guard.fail_upload_task(order_index); - Err(e) - } - } - } - - pub fn read_filter( - self: &LocalVersionManager, - read_epoch: HummockEpoch, - table_id: TableId, - table_key_range: &R, - ) -> ReadVersion - where - R: RangeBounds>, - B: AsRef<[u8]>, - { - LocalVersion::read_filter(&self.local_version, read_epoch, table_id, table_key_range) - } - - pub fn get_pinned_version(&self) -> PinnedVersion { - self.local_version.read().pinned_version().clone() - } - - pub(crate) fn buffer_tracker(&self) -> &BufferTracker { - &self.buffer_tracker - } - - pub fn sstable_id_manager(&self) -> SstableIdManagerRef { - self.sstable_id_manager.clone() - } -} - -// concurrent worker thread of `LocalVersionManager` -impl LocalVersionManager { - pub fn clear_shared_buffer(&self) { - self.local_version.write().shared_buffer.clear(); - self.local_version.write().sync_uncommitted_data.clear(); - } -} - -#[cfg(any(test, feature = "test"))] -// Some method specially for tests of `LocalVersionManager` -impl LocalVersionManager { - pub fn local_version(&self) -> &RwLock { - &self.local_version - } - - pub fn get_local_version(&self) -> LocalVersion { - self.local_version.read().clone() - } - - pub fn get_shared_buffer_size(&self) -> usize { - self.buffer_tracker.get_buffer_size() - } -} diff --git a/src/storage/src/hummock/local_version/mod.rs b/src/storage/src/hummock/local_version/mod.rs index 9c6da23568b9f..b01889674c302 100644 --- a/src/storage/src/hummock/local_version/mod.rs +++ b/src/storage/src/hummock/local_version/mod.rs @@ -20,7 +20,6 @@ use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::shared_buffer::{OrderSortedUncommittedData, SharedBuffer}; pub mod local_version_impl; -pub mod local_version_manager; pub mod pinned_version; #[derive(Clone)] diff --git a/src/storage/src/hummock/mod.rs b/src/storage/src/hummock/mod.rs index 4b577640f273e..33d4c2c7d3c07 100644 --- a/src/storage/src/hummock/mod.rs +++ b/src/storage/src/hummock/mod.rs @@ -29,7 +29,6 @@ use risingwave_pb::hummock::HummockVersion; use risingwave_pb::hummock::{version_update_payload, SstableInfo}; use risingwave_rpc_client::HummockMetaClient; use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; -use tokio::sync::watch; use tracing::log::error; mod block_cache; @@ -55,7 +54,6 @@ pub mod iterator; pub mod shared_buffer; pub mod sstable_store; mod state_store; -mod state_store_v1; #[cfg(any(test, feature = "test"))] pub mod test_utils; pub mod utils; @@ -70,7 +68,6 @@ mod validator; pub mod value; pub use error::*; -use local_version::local_version_manager::{LocalVersionManager, LocalVersionManagerRef}; pub use risingwave_common::cache::{CacheableEntry, LookupResult, LruCache}; use risingwave_common_service::observer_manager::{NotificationClient, ObserverManager}; use risingwave_hummock_sdk::filter_key_extractor::{ @@ -84,7 +81,9 @@ use self::iterator::{BackwardUserIterator, HummockIterator, UserIterator}; pub use self::sstable_store::*; use super::monitor::HummockStateStoreMetrics; use crate::error::StorageResult; -use crate::hummock::backup_reader::{BackupReader, BackupReaderRef}; +#[cfg(any(test, feature = "test"))] +use crate::hummock::backup_reader::BackupReader; +use crate::hummock::backup_reader::BackupReaderRef; use crate::hummock::compactor::CompactorContext; use crate::hummock::event_handler::hummock_event_handler::BufferTracker; use crate::hummock::event_handler::{HummockEvent, HummockEventHandler}; @@ -494,163 +493,6 @@ pub fn get_from_batch( }) } -#[derive(Clone)] -pub struct HummockStorageV1 { - options: Arc, - - local_version_manager: LocalVersionManagerRef, - - sstable_store: SstableStoreRef, - - /// Statistics - state_store_metrics: Arc, - - sstable_id_manager: SstableIdManagerRef, - - filter_key_extractor_manager: FilterKeyExtractorManagerRef, - - hummock_event_sender: UnboundedSender, - - _shutdown_guard: Arc, - - version_update_notifier_tx: Arc>, - - tracing: Arc, -} - -impl HummockStorageV1 { - /// Creates a [`HummockStorageV1`]. - pub async fn new( - options: Arc, - sstable_store: SstableStoreRef, - hummock_meta_client: Arc, - notification_client: impl NotificationClient, - // TODO: separate `HummockStats` from `HummockStateStoreMetrics`. - state_store_metrics: Arc, - tracing: Arc, - compactor_metrics: Arc, - ) -> HummockResult { - // For conflict key detection. Enabled by setting `write_conflict_detection_enabled` to - // true in `StorageConfig` - let sstable_id_manager = Arc::new(SstableIdManager::new( - hummock_meta_client.clone(), - options.sstable_id_remote_fetch_number, - )); - - let filter_key_extractor_manager = Arc::new(FilterKeyExtractorManager::default()); - let (event_tx, mut event_rx) = unbounded_channel(); - let backup_manager = BackupReader::unused(); - let observer_manager = ObserverManager::new( - notification_client, - HummockObserverNode::new( - filter_key_extractor_manager.clone(), - backup_manager, - event_tx.clone(), - ), - ) - .await; - observer_manager.start().await; - - let hummock_version = match event_rx.recv().await { - Some(HummockEvent::VersionUpdate(version_update_payload::Payload::PinnedVersion(version))) => version, - _ => unreachable!("the hummock observer manager is the first one to take the event tx. Should be full hummock version") - }; - - let (pin_version_tx, pin_version_rx) = unbounded_channel(); - let pinned_version = PinnedVersion::new(hummock_version, pin_version_tx); - tokio::spawn(start_pinned_version_worker( - pin_version_rx, - hummock_meta_client.clone(), - )); - - let compactor_context = Arc::new(CompactorContext::new_local_compact_context( - options.clone(), - sstable_store.clone(), - hummock_meta_client.clone(), - compactor_metrics.clone(), - sstable_id_manager.clone(), - filter_key_extractor_manager.clone(), - CompactorRuntimeConfig::default(), - )); - - let buffer_tracker = BufferTracker::from_storage_opts(&options); - - let local_version_manager = - LocalVersionManager::new(pinned_version.clone(), compactor_context, buffer_tracker); - - let local_version_manager_clone = local_version_manager.clone(); - let (epoch_update_tx, _) = watch::channel(pinned_version.max_committed_epoch()); - let epoch_update_tx = Arc::new(epoch_update_tx); - let epoch_update_tx_clone = epoch_update_tx.clone(); - - tokio::spawn(async move { - while let Some(event) = event_rx.recv().await { - match event { - HummockEvent::Shutdown => { - break; - } - HummockEvent::VersionUpdate(version_update) => { - local_version_manager_clone.try_update_pinned_version(version_update); - // TODO: this is - epoch_update_tx.send_replace( - local_version_manager_clone - .get_pinned_version() - .max_committed_epoch(), - ); - } - _ => { - unreachable!("for hummock v1, there should only be shutdown and version update event"); - } - } - } - }); - - let instance = Self { - options, - local_version_manager, - sstable_store, - state_store_metrics, - sstable_id_manager, - filter_key_extractor_manager, - _shutdown_guard: Arc::new(HummockStorageShutdownGuard { - shutdown_sender: event_tx.clone(), - }), - version_update_notifier_tx: epoch_update_tx_clone, - hummock_event_sender: event_tx, - tracing, - }; - - Ok(instance) - } - - pub fn options(&self) -> &Arc { - &self.options - } - - pub fn sstable_store(&self) -> SstableStoreRef { - self.sstable_store.clone() - } - - pub fn sstable_id_manager(&self) -> &SstableIdManagerRef { - &self.sstable_id_manager - } - - pub fn filter_key_extractor_manager(&self) -> &FilterKeyExtractorManagerRef { - &self.filter_key_extractor_manager - } - - pub fn get_memory_limiter(&self) -> Arc { - self.local_version_manager - .buffer_tracker() - .get_memory_limiter() - .clone() - } - - pub fn get_pinned_version(&self) -> PinnedVersion { - self.local_version_manager.get_pinned_version() - } -} - pub(crate) trait HummockIteratorType: 'static { type Direction: HummockIteratorDirection; type SstableIteratorType: SstableIteratorType; diff --git a/src/storage/src/hummock/sstable/builder.rs b/src/storage/src/hummock/sstable/builder.rs index 313c7fee698c7..2c87f503f50ec 100644 --- a/src/storage/src/hummock/sstable/builder.rs +++ b/src/storage/src/hummock/sstable/builder.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cmp; use std::collections::BTreeSet; use std::sync::Arc; @@ -115,7 +116,11 @@ pub struct SstableBuilder { /// `last_table_stats` accumulates stats for `last_table_id` and finalizes it in `table_stats` /// by `finalize_last_table_stats` last_table_stats: TableStats, + filter_builder: F, + + min_epoch: u64, + max_epoch: u64, } impl SstableBuilder { @@ -163,6 +168,8 @@ impl SstableBuilder { total_key_count: 0, table_stats: Default::default(), last_table_stats: Default::default(), + min_epoch: u64::MAX, + max_epoch: u64::MIN, } } @@ -256,6 +263,9 @@ impl SstableBuilder { self.raw_key.clear(); self.raw_value.clear(); + self.min_epoch = cmp::min(self.min_epoch, full_key.epoch); + self.max_epoch = cmp::max(self.max_epoch, full_key.epoch); + if self.block_builder.approximate_len() >= self.options.block_capacity { self.build_block().await?; } @@ -333,6 +343,20 @@ impl SstableBuilder { range_tombstone_list: self.range_tombstones, }; meta.estimated_size = meta.encoded_size() as u32 + meta_offset as u32; + + // Expand the epoch of the whole sst by tombstone epoch + let (tombstone_min_epoch, tombstone_max_epoch) = { + let mut tombstone_min_epoch = u64::MAX; + let mut tombstone_max_epoch = u64::MIN; + + for tombstone in &meta.range_tombstone_list { + tombstone_min_epoch = cmp::min(tombstone_min_epoch, tombstone.sequence); + tombstone_max_epoch = cmp::max(tombstone_max_epoch, tombstone.sequence); + } + + (tombstone_min_epoch, tombstone_max_epoch) + }; + let sst_info = SstableInfo { id: self.sstable_id, key_range: Some(risingwave_pb::hummock::KeyRange { @@ -346,12 +370,17 @@ impl SstableBuilder { stale_key_count: self.stale_key_count, total_key_count: self.total_key_count, divide_version: 0, + min_epoch: cmp::min(self.min_epoch, tombstone_min_epoch), + max_epoch: cmp::max(self.max_epoch, tombstone_max_epoch), }; tracing::trace!( - "meta_size {} bloom_filter_size {} add_key_counts {} ", + "meta_size {} bloom_filter_size {} add_key_counts {} stale_key_count {} min_epoch {} max_epoch {}", meta.encoded_size(), meta.bloom_filter.len(), self.total_key_count, + self.stale_key_count, + self.min_epoch, + self.max_epoch, ); let bloom_filter_size = meta.bloom_filter.len(); let (avg_key_size, avg_value_size) = if self.table_stats.is_empty() { diff --git a/src/storage/src/hummock/sstable/mod.rs b/src/storage/src/hummock/sstable/mod.rs index d10574cb41dba..a1f7d8bb5b9a2 100644 --- a/src/storage/src/hummock/sstable/mod.rs +++ b/src/storage/src/hummock/sstable/mod.rs @@ -193,7 +193,7 @@ impl Sstable { #[inline] pub fn estimate_size(&self) -> usize { - 8 /* id */ + self.meta.encoded_size() + 8 /* id */ + self.filter_reader.estimate_size() + self.meta.encoded_size() } #[cfg(test)] @@ -211,6 +211,8 @@ impl Sstable { stale_key_count: 0, total_key_count: self.meta.key_count as u64, divide_version: 0, + min_epoch: 0, + max_epoch: 0, } } } diff --git a/src/storage/src/hummock/sstable/xor_filter.rs b/src/storage/src/hummock/sstable/xor_filter.rs index 5c981c287866c..ea6a678b27107 100644 --- a/src/storage/src/hummock/sstable/xor_filter.rs +++ b/src/storage/src/hummock/sstable/xor_filter.rs @@ -103,6 +103,10 @@ impl XorFilterReader { } } + pub fn estimate_size(&self) -> usize { + self.filter.fingerprints.len() * std::mem::size_of::() + } + pub fn is_empty(&self) -> bool { self.filter.block_length == 0 } diff --git a/src/storage/src/hummock/sstable_store.rs b/src/storage/src/hummock/sstable_store.rs index 36f2c7ae47459..272764a83f4f5 100644 --- a/src/storage/src/hummock/sstable_store.rs +++ b/src/storage/src/hummock/sstable_store.rs @@ -346,7 +346,7 @@ impl SstableStore { .map_err(HummockError::object_io_error)?; let meta = SstableMeta::decode(&mut &buf[..])?; let sst = Sstable::new(sst_id, meta); - let charge = sst.meta.encoded_size(); + let charge = sst.estimate_size(); let add = (now.elapsed().as_secs_f64() * 1000.0).ceil(); stats_ptr.fetch_add(add as u64, Ordering::Relaxed); Ok((Box::new(sst), charge)) diff --git a/src/storage/src/hummock/state_store_v1.rs b/src/storage/src/hummock/state_store_v1.rs deleted file mode 100644 index ee063a2c3434f..0000000000000 --- a/src/storage/src/hummock/state_store_v1.rs +++ /dev/null @@ -1,638 +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 std::cmp::Ordering; -use std::future::Future; -use std::ops::{Bound, RangeBounds}; -use std::sync::Arc; -use std::time::Duration; - -use bytes::Bytes; -use futures::TryFutureExt; -use itertools::Itertools; -use minitrace::future::FutureExt; -use minitrace::Span; -use risingwave_common::catalog::TableId; -use risingwave_common::util::epoch::INVALID_EPOCH; -use risingwave_hummock_sdk::key::{ - bound_table_key_range, map_table_key_range, user_key, FullKey, TableKey, TableKeyRange, UserKey, -}; -use risingwave_hummock_sdk::key_range::KeyRangeCommon; -use risingwave_hummock_sdk::HummockReadEpoch; -use risingwave_pb::hummock::LevelType; -use tokio::sync::oneshot; -use tracing::log::warn; - -use super::iterator::{ - ConcatIteratorInner, DirectedUserIterator, DirectionEnum, HummockIteratorUnion, -}; -use super::utils::validate_safe_epoch; -use super::{ - get_from_order_sorted_uncommitted_data, get_from_sstable_info, hit_sstable_bloom_filter, - HummockStorageV1, SstableIteratorType, -}; -use crate::error::{StorageError, StorageResult}; -use crate::hummock::event_handler::HummockEvent; -use crate::hummock::iterator::{ - DirectedUserIteratorBuilder, ForwardMergeRangeIterator, HummockIteratorDirection, -}; -use crate::hummock::local_version::ReadVersion; -use crate::hummock::shared_buffer::build_ordered_merge_iter; -use crate::hummock::sstable::SstableIteratorReadOptions; -use crate::hummock::utils::{prune_nonoverlapping_ssts, prune_overlapping_ssts}; -use crate::hummock::{ - DeleteRangeAggregator, ForwardIter, HummockEpoch, HummockError, HummockIteratorType, - HummockResult, Sstable, -}; -use crate::monitor::{ - GetLocalMetricsGuard, HummockStateStoreMetrics, IterLocalMetricsGuard, StoreLocalStatistic, -}; -use crate::storage_value::StorageValue; -use crate::store::*; -use crate::{ - define_local_state_store_associated_type, define_state_store_associated_type, - define_state_store_read_associated_type, define_state_store_write_associated_type, -}; - -impl HummockStorageV1 { - /// Gets the value of a specified `key`. - /// The result is based on a snapshot corresponding to the given `epoch`. - /// if `key` has consistent hash virtual node value, then such value is stored in `value_meta` - /// - /// If `Ok(Some())` is returned, the key is found. If `Ok(None)` is returned, - /// the key is not found. If `Err()` is returned, the searching for the key - /// failed due to other non-EOF errors. - async fn get<'a>( - &'a self, - table_key: TableKey<&'a [u8]>, - epoch: HummockEpoch, - read_options: ReadOptions, - ) -> StorageResult> { - let table_id = read_options.table_id; - let mut stats_guard = - GetLocalMetricsGuard::new(self.state_store_metrics.clone(), read_options.table_id); - stats_guard.local_stats.found_key = true; - let ReadVersion { - shared_buffer_data, - pinned_version, - sync_uncommitted_data, - } = self.read_filter(epoch, &read_options, &(table_key..=table_key))?; - - let full_key = FullKey::new(table_id, table_key, epoch); - - // Query shared buffer. Return the value without iterating SSTs if found - for uncommitted_data in shared_buffer_data { - // iterate over uncommitted data in order index in descending order - let (value, table_count) = get_from_order_sorted_uncommitted_data( - self.sstable_store.clone(), - uncommitted_data, - full_key, - &mut stats_guard.local_stats, - &read_options, - ) - .await?; - if let Some(v) = value { - return Ok(v.into_user_value()); - } - stats_guard.local_stats.sub_iter_count += table_count as u64; - } - for sync_uncommitted_data in sync_uncommitted_data { - let (value, table_count) = get_from_order_sorted_uncommitted_data( - self.sstable_store.clone(), - sync_uncommitted_data, - full_key, - &mut stats_guard.local_stats, - &read_options, - ) - .await?; - if let Some(v) = value { - return Ok(v.into_user_value()); - } - stats_guard.local_stats.sub_iter_count += table_count as u64; - } - - let dist_key_hash = read_options.prefix_hint.as_ref().map(|dist_key| { - Sstable::hash_for_bloom_filter(dist_key.as_ref(), read_options.table_id.table_id()) - }); - - // Because SST meta records encoded key range, - // the filter key needs to be encoded as well. - let encoded_user_key = UserKey::new(read_options.table_id, table_key).encode(); - // See comments in HummockStorage::iter_inner for details about using compaction_group_id in - // read/write path. - assert!(pinned_version.is_valid()); - for level in pinned_version.levels(table_id) { - if level.table_infos.is_empty() { - continue; - } - match level.level_type() { - LevelType::Overlapping | LevelType::Unspecified => { - let single_table_key_range = table_key..=table_key; - let sstable_infos = prune_overlapping_ssts( - &level.table_infos, - table_id, - &single_table_key_range, - ); - for sstable_info in sstable_infos { - stats_guard.local_stats.sub_iter_count += 1; - if let Some(v) = get_from_sstable_info( - self.sstable_store.clone(), - sstable_info, - full_key, - &read_options, - dist_key_hash, - &mut stats_guard.local_stats, - ) - .await? - { - return Ok(v.into_user_value()); - } - } - } - LevelType::Nonoverlapping => { - let mut table_info_idx = level.table_infos.partition_point(|table| { - let ord = user_key(&table.key_range.as_ref().unwrap().left) - .cmp(encoded_user_key.as_ref()); - ord == Ordering::Less || ord == Ordering::Equal - }); - if table_info_idx == 0 { - continue; - } - table_info_idx = table_info_idx.saturating_sub(1); - let ord = level.table_infos[table_info_idx] - .key_range - .as_ref() - .unwrap() - .compare_right_with_user_key(&encoded_user_key); - // the case that the key falls into the gap between two ssts - if ord == Ordering::Less { - continue; - } - - stats_guard.local_stats.sub_iter_count += 1; - if let Some(v) = get_from_sstable_info( - self.sstable_store.clone(), - &level.table_infos[table_info_idx], - full_key, - &read_options, - dist_key_hash, - &mut stats_guard.local_stats, - ) - .await? - { - return Ok(v.into_user_value()); - } - } - } - } - stats_guard.local_stats.found_key = false; - Ok(None) - } - - fn read_filter( - &self, - epoch: HummockEpoch, - read_options: &ReadOptions, - table_key_range: &R, - ) -> HummockResult - where - R: RangeBounds>, - B: AsRef<[u8]>, - { - let read_version = - self.local_version_manager - .read_filter(epoch, read_options.table_id, table_key_range); - - // Check epoch validity - validate_safe_epoch(read_version.pinned_version.safe_epoch(), epoch)?; - - Ok(read_version) - } - - async fn iter_inner( - &self, - epoch: HummockEpoch, - table_key_range: TableKeyRange, - read_options: ReadOptions, - ) -> StorageResult - where - T: HummockIteratorType, - { - let table_id = read_options.table_id; - let min_epoch = gen_min_epoch(epoch, read_options.retention_seconds.as_ref()); - let iter_read_options = Arc::new(SstableIteratorReadOptions::default()); - let mut overlapped_iters = vec![]; - let user_key_range = bound_table_key_range(table_id, &table_key_range); - - let ReadVersion { - shared_buffer_data, - pinned_version, - sync_uncommitted_data, - } = self.read_filter(epoch, &read_options, &table_key_range)?; - - let mut local_stats = StoreLocalStatistic::default(); - for uncommitted_data in shared_buffer_data { - overlapped_iters.push(HummockIteratorUnion::Second( - build_ordered_merge_iter::( - &uncommitted_data, - self.sstable_store.clone(), - &mut local_stats, - iter_read_options.clone(), - ) - .in_span(Span::enter_with_local_parent( - "build_ordered_merge_iter_shared_buffer", - )) - .await?, - )); - } - for sync_uncommitted_data in sync_uncommitted_data { - overlapped_iters.push(HummockIteratorUnion::Second( - build_ordered_merge_iter::( - &sync_uncommitted_data, - self.sstable_store.clone(), - &mut local_stats, - iter_read_options.clone(), - ) - .in_span(Span::enter_with_local_parent( - "build_ordered_merge_iter_uncommitted", - )) - .await?, - )) - } - - local_stats.staging_imm_iter_count = overlapped_iters.len() as u64; - // Generate iterators for versioned ssts by filter out ssts that do not overlap with the - // user key range derived from the given `table_key_range` and `table_id`. - - // The correctness of using compaction_group_id in read path and write path holds only - // because we are currently: - // - // a) adopting static compaction group. It means a table_id->compaction_group mapping would - // never change since creation until the table is dropped. - // - // b) enforcing shared buffer to split output SSTs by compaction group. It means no SSTs - // would contain tables from different compaction_group, even for those in L0. - // - // When adopting dynamic compaction group in the future, be sure to revisit this assumption. - - // Because SST meta records encoded key range, - // the filter key range needs to be encoded as well. - let encoded_user_key_range = ( - user_key_range.0.as_ref().map(UserKey::encode), - user_key_range.1.as_ref().map(UserKey::encode), - ); - assert!(pinned_version.is_valid()); - // encode once - let bloom_filter_prefix_hash = read_options - .prefix_hint - .as_ref() - .map(|hint| Sstable::hash_for_bloom_filter(hint, read_options.table_id.table_id())); - for level in pinned_version.levels(table_id) { - if level.table_infos.is_empty() { - continue; - } - if level.level_type == LevelType::Nonoverlapping as i32 { - let matched_table_infos = - prune_nonoverlapping_ssts(&level.table_infos, &encoded_user_key_range); - - let pruned_sstables = match T::Direction::direction() { - DirectionEnum::Backward => matched_table_infos.rev().collect_vec(), - DirectionEnum::Forward => matched_table_infos.collect_vec(), - }; - - let mut sstables = vec![]; - for sstable_info in pruned_sstables { - if let Some(prefix_hash) = bloom_filter_prefix_hash.as_ref() { - let sstable = self - .sstable_store - .sstable(sstable_info, &mut local_stats) - .in_span(Span::enter_with_local_parent("get_sstable")) - .await?; - - if hit_sstable_bloom_filter(sstable.value(), *prefix_hash, &mut local_stats) - { - sstables.push((*sstable_info).clone()); - } - } else { - sstables.push((*sstable_info).clone()); - } - } - - overlapped_iters.push(HummockIteratorUnion::Third(ConcatIteratorInner::< - T::SstableIteratorType, - >::new( - sstables, - self.sstable_store(), - iter_read_options.clone(), - ))); - } else { - let table_infos = - prune_overlapping_ssts(&level.table_infos, table_id, &table_key_range); - for table_info in table_infos.into_iter().rev() { - let sstable = self - .sstable_store - .sstable(table_info, &mut local_stats) - .in_span(Span::enter_with_local_parent("get_sstable")) - .await?; - if let Some(prefix_hash) = bloom_filter_prefix_hash.as_ref() { - if !hit_sstable_bloom_filter( - sstable.value(), - *prefix_hash, - &mut local_stats, - ) { - continue; - } - } - - overlapped_iters.push(HummockIteratorUnion::Fourth( - T::SstableIteratorType::create( - sstable, - self.sstable_store(), - iter_read_options.clone(), - ), - )); - } - } - } - - local_stats.sub_iter_count = overlapped_iters.len() as u64; - - // TODO: implement delete range if the code of this file would not be delete. - let delete_range_iter = ForwardMergeRangeIterator::default(); - let delete_range_agg = DeleteRangeAggregator::new(delete_range_iter, epoch); - // The input of the user iterator is a `HummockIteratorUnion` of 4 different types. We use - // the union because the underlying merge iterator - let mut user_iterator = T::UserIteratorBuilder::create( - overlapped_iters, - user_key_range, - epoch, - min_epoch, - Some(pinned_version), - delete_range_agg, - ); - - user_iterator - .rewind() - .in_span(Span::enter_with_local_parent("rewind")) - .await?; - - local_stats.found_key = user_iterator.is_valid(); - Ok(HummockStateStoreIter::new( - user_iterator, - self.state_store_metrics.clone(), - read_options.table_id, - local_stats, - )) - } -} - -impl StateStoreRead for HummockStorageV1 { - type IterStream = StreamTypeOfIter; - - define_state_store_read_associated_type!(); - - fn get<'a>( - &'a self, - key: &'a [u8], - epoch: HummockEpoch, - read_options: ReadOptions, - ) -> Self::GetFuture<'_> { - self.get(TableKey(key), epoch, read_options) - } - - /// Returns an iterator that scan from the begin key to the end key - /// The result is based on a snapshot corresponding to the given `epoch`. - fn iter( - &self, - key_range: (Bound>, Bound>), - epoch: HummockEpoch, - read_options: ReadOptions, - ) -> Self::IterFuture<'_> { - self.iter_inner::(epoch, map_table_key_range(key_range), read_options) - .map_ok(|iter| iter.into_stream()) - .in_span(self.tracing.new_tracer("hummock_iter")) - } -} - -impl StateStoreWrite for HummockStorageV1 { - define_state_store_write_associated_type!(); - - /// Writes a batch to storage. The batch should be: - /// * Ordered. KV pairs will be directly written to the table, so it must be ordered. - /// * Locally unique. There should not be two or more operations on the same key in one write - /// batch. - fn ingest_batch( - &self, - kv_pairs: Vec<(Bytes, StorageValue)>, - delete_ranges: Vec<(Bytes, Bytes)>, - write_options: WriteOptions, - ) -> Self::IngestBatchFuture<'_> { - async move { - if kv_pairs.is_empty() { - return Ok(0); - } - - let epoch = write_options.epoch; - // See comments in HummockStorage::iter_inner for details about using - // compaction_group_id in read/write path. - let size = self - .local_version_manager - .write_shared_buffer(epoch, kv_pairs, delete_ranges, write_options.table_id) - .await?; - Ok(size) - } - } -} - -impl LocalStateStore for HummockStorageV1 { - define_local_state_store_associated_type!(); - - fn may_exist( - &self, - _key_range: (Bound>, Bound>), - _read_options: ReadOptions, - ) -> Self::MayExistFuture<'_> { - async move { Ok(true) } - } -} - -impl StateStore for HummockStorageV1 { - type Local = Self; - - type NewLocalFuture<'a> = impl Future + Send + 'a; - - define_state_store_associated_type!(); - - /// Waits until the local hummock version contains the epoch. If `wait_epoch` is `Current`, - /// we will only check whether it is le `sealed_epoch` and won't wait. - fn try_wait_epoch(&self, wait_epoch: HummockReadEpoch) -> Self::WaitEpochFuture<'_> { - async move { - // Ok(self.local_version_manager.try_wait_epoch(epoch).await?) - let wait_epoch = match wait_epoch { - HummockReadEpoch::Committed(epoch) => epoch, - HummockReadEpoch::Current(epoch) => { - // let sealed_epoch = self.local_version.read().get_sealed_epoch(); - let sealed_epoch = self - .local_version_manager - .local_version - .read() - .get_sealed_epoch(); - assert!( - epoch <= sealed_epoch - && epoch != HummockEpoch::MAX - , - "current epoch can't read, because the epoch in storage is not updated, epoch{}, sealed epoch{}" - ,epoch - ,sealed_epoch - ); - return Ok(()); - } - HummockReadEpoch::NoWait(_) | HummockReadEpoch::Backup(_) => return Ok(()), - }; - if wait_epoch == HummockEpoch::MAX { - panic!("epoch should not be u64::MAX"); - } - - let mut receiver = self.version_update_notifier_tx.subscribe(); - // avoid unnecessary check in the loop if the value does not change - let max_committed_epoch = *receiver.borrow_and_update(); - if max_committed_epoch >= wait_epoch { - return Ok(()); - } - loop { - match tokio::time::timeout(Duration::from_secs(30), receiver.changed()).await { - Err(elapsed) => { - // The reason that we need to retry here is batch scan in - // chain/rearrange_chain is waiting for an - // uncommitted epoch carried by the CreateMV barrier, which - // can take unbounded time to become committed and propagate - // to the CN. We should consider removing the retry as well as wait_epoch - // for chain/rearrange_chain if we enforce - // chain/rearrange_chain to be scheduled on the same - // CN with the same distribution as the upstream MV. - // See #3845 for more details. - tracing::warn!( - "wait_epoch {:?} timeout when waiting for version update elapsed {:?}s", - wait_epoch, - elapsed - ); - continue; - } - Ok(Err(_)) => { - return StorageResult::Err(StorageError::Hummock( - HummockError::wait_epoch("tx dropped"), - )); - } - Ok(Ok(_)) => { - let max_committed_epoch = *receiver.borrow(); - if max_committed_epoch >= wait_epoch { - return Ok(()); - } - } - } - } - } - } - - fn sync(&self, epoch: u64) -> Self::SyncFuture<'_> { - async move { - if epoch == INVALID_EPOCH { - warn!("syncing invalid epoch"); - return Ok(SyncResult { - sync_size: 0, - uncommitted_ssts: vec![], - }); - } - self.local_version_manager - .await_sync_shared_buffer(epoch) - .await - .map_err(StorageError::Hummock) - } - } - - fn seal_epoch(&self, epoch: u64, is_checkpoint: bool) { - if epoch == INVALID_EPOCH { - warn!("sealing invalid epoch"); - return; - } - self.local_version_manager.seal_epoch(epoch, is_checkpoint); - } - - fn clear_shared_buffer(&self) -> Self::ClearSharedBufferFuture<'_> { - async move { - let (tx, rx) = oneshot::channel(); - self.hummock_event_sender - .send(HummockEvent::Clear(tx)) - .expect("should send success"); - rx.await.expect("should wait success"); - Ok(()) - } - } - - fn new_local(&self, _table_id: TableId) -> Self::NewLocalFuture<'_> { - async { self.clone() } - } - - fn validate_read_epoch(&self, _epoch: HummockReadEpoch) -> StorageResult<()> { - // Returns Ok directly, since removal of HummockStorageV1 is planned. - Ok(()) - } -} - -pub struct HummockStateStoreIter { - inner: DirectedUserIterator, - stats_guard: IterLocalMetricsGuard, -} - -impl HummockStateStoreIter { - #[allow(dead_code)] - fn new( - inner: DirectedUserIterator, - metrics: Arc, - table_id: TableId, - local_stats: StoreLocalStatistic, - ) -> Self { - Self { - inner, - stats_guard: IterLocalMetricsGuard::new(metrics, table_id, local_stats), - } - } -} - -impl StateStoreIter for HummockStateStoreIter { - type Item = StateStoreIterItem; - - type NextFuture<'a> = impl StateStoreIterNextFutureTrait<'a>; - - fn next(&mut self) -> Self::NextFuture<'_> { - async move { - let iter = &mut self.inner; - - if iter.is_valid() { - let kv = (iter.key().clone(), iter.value().clone()); - iter.next().await?; - Ok(Some(kv)) - } else { - Ok(None) - } - } - } -} - -impl Drop for HummockStateStoreIter { - fn drop(&mut self) { - self.inner - .collect_local_statistic(&mut self.stats_guard.local_stats); - } -} diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 5a71706d0a463..fde7ab4aad4e8 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -555,11 +555,6 @@ impl HummockVersionReader { .sstable(sstable_info, &mut local_stats) .in_span(Span::enter_with_local_parent("get_sstable")) .await?; - if let Some(prefix_hash) = bloom_filter_prefix_hash.as_ref() { - if !hit_sstable_bloom_filter(table_holder.value(), *prefix_hash, &mut local_stats) { - continue; - } - } if !table_holder.value().meta.range_tombstone_list.is_empty() && !read_options.ignore_range_tombstone @@ -567,6 +562,12 @@ impl HummockVersionReader { delete_range_iter .add_sst_iter(SstableDeleteRangeIterator::new(table_holder.clone())); } + if let Some(prefix_hash) = bloom_filter_prefix_hash.as_ref() { + if !hit_sstable_bloom_filter(table_holder.value(), *prefix_hash, &mut local_stats) { + continue; + } + } + staging_sst_iter_count += 1; staging_iters.push(HummockIteratorUnion::Second(SstableIterator::new( table_holder, diff --git a/src/storage/src/hummock/test_utils.rs b/src/storage/src/hummock/test_utils.rs index dbe8530da2e52..edc733b488dfe 100644 --- a/src/storage/src/hummock/test_utils.rs +++ b/src/storage/src/hummock/test_utils.rs @@ -110,6 +110,8 @@ pub fn gen_dummy_sst_info( stale_key_count: 0, total_key_count: 0, divide_version: 0, + min_epoch: 0, + max_epoch: 0, } } @@ -182,6 +184,8 @@ pub async fn put_sst( stale_key_count: 0, total_key_count: 0, divide_version: 0, + min_epoch: 0, + max_epoch: 0, }; let writer_output = writer.finish(meta).await?; writer_output.await.unwrap()?; diff --git a/src/storage/src/opts.rs b/src/storage/src/opts.rs index 83b721445c5ea..9ee84f608ed9a 100644 --- a/src/storage/src/opts.rs +++ b/src/storage/src/opts.rs @@ -13,8 +13,8 @@ // limitations under the License. use risingwave_common::config::RwConfig; +use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_pb::meta::SystemParams; -use risingwave_rpc_client::SystemParamsReader; #[derive(Clone, Debug)] pub struct StorageOpts { @@ -55,7 +55,6 @@ pub struct StorageOpts { /// Max sub compaction task numbers pub max_sub_compaction: u32, pub max_concurrent_compaction_task_number: u64, - pub enable_state_store_v1: bool, pub file_cache_dir: String, pub file_cache_capacity_mb: usize, @@ -112,7 +111,6 @@ impl From<(&RwConfig, &SystemParamsReader)> for StorageOpts { 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, - enable_state_store_v1: c.storage.enable_state_store_v1, file_cache_dir: c.storage.file_cache.dir.clone(), file_cache_capacity_mb: c.storage.file_cache.capacity_mb, file_cache_total_buffer_capacity_mb: c.storage.file_cache.total_buffer_capacity_mb, diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index 24e6ced30f2a4..a2243c7cbb65e 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -27,8 +27,8 @@ use crate::hummock::backup_reader::{parse_meta_snapshot_storage, BackupReader}; use crate::hummock::hummock_meta_client::MonitoredHummockMetaClient; use crate::hummock::sstable_store::SstableStoreRef; use crate::hummock::{ - HummockStorage, HummockStorageV1, MemoryLimiter, SstableIdManagerRef, SstableStore, - TieredCache, TieredCacheMetricsBuilder, + HummockStorage, MemoryLimiter, SstableIdManagerRef, SstableStore, TieredCache, + TieredCacheMetricsBuilder, }; use crate::memory::sled::SledStateStore; use crate::memory::MemoryStateStore; @@ -40,12 +40,12 @@ use crate::opts::StorageOpts; use crate::StateStore; pub type HummockStorageType = impl StateStore + AsHummockTrait; -pub type HummockStorageV1Type = impl StateStore + AsHummockTrait; pub type MemoryStateStoreType = impl StateStore + AsHummockTrait; pub type SledStateStoreType = impl StateStore + AsHummockTrait; /// The type erased [`StateStore`]. #[derive(Clone, EnumAsInner)] +#[allow(clippy::enum_variant_names)] pub enum StateStoreImpl { /// The Hummock state store, which operates on an S3-like service. URLs beginning with /// `hummock` will be automatically recognized as Hummock state store. @@ -56,7 +56,6 @@ pub enum StateStoreImpl { /// * `hummock+minio://KEY:SECRET@minio-ip:port` /// * `hummock+memory` (should only be used in 1 compute node mode) HummockStateStore(Monitored), - HummockStateStoreV1(Monitored), /// In-memory B-Tree state store. Should only be used in unit and integration tests. If you /// want speed up e2e test, you should use Hummock in-memory mode instead. Also, this state /// store misses some critical implementation to ensure the correctness of persisting streaming @@ -124,16 +123,6 @@ impl StateStoreImpl { ) } - pub fn hummock_v1( - state_store: HummockStorageV1, - storage_metrics: Arc, - ) -> Self { - // The specific type of HummockStateStoreV1Type in deducted here. - Self::HummockStateStoreV1( - may_dynamic_dispatch(may_verify(state_store)).monitored(storage_metrics), - ) - } - pub fn sled( state_store: SledStateStore, storage_metrics: Arc, @@ -161,12 +150,6 @@ impl StateStoreImpl { .as_hummock_trait() .expect("should be hummock"), ), - StateStoreImpl::HummockStateStoreV1(hummock) => Some( - hummock - .inner() - .as_hummock_trait() - .expect("should be hummock"), - ), _ => None, } } @@ -191,7 +174,6 @@ impl Debug for StateStoreImpl { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { StateStoreImpl::HummockStateStore(_) => write!(f, "HummockStateStore"), - StateStoreImpl::HummockStateStoreV1(_) => write!(f, "HummockStateStoreV1"), StateStoreImpl::MemoryStateStore(_) => write!(f, "MemoryStateStore"), StateStoreImpl::SledStateStore(_) => write!(f, "SledStateStore"), } @@ -233,8 +215,6 @@ macro_rules! dispatch_state_store { } StateStoreImpl::HummockStateStore($store) => $body, - - StateStoreImpl::HummockStateStoreV1($store) => $body, } }}; } @@ -530,40 +510,25 @@ impl StateStoreImpl { let notification_client = RpcNotificationClient::new(hummock_meta_client.get_inner().clone()); - if !opts.enable_state_store_v1 { - let backup_store = parse_meta_snapshot_storage( - &opts.backup_storage_url, - &opts.backup_storage_directory, - ) - .await?; - let backup_reader = BackupReader::new(backup_store); - let inner = HummockStorage::new( - opts.clone(), - sstable_store, - backup_reader, - hummock_meta_client.clone(), - notification_client, - state_store_metrics.clone(), - tracing, - compactor_metrics.clone(), - ) - .await?; - - StateStoreImpl::hummock(inner, storage_metrics) - } else { - let inner = HummockStorageV1::new( - opts.clone(), - sstable_store, - hummock_meta_client.clone(), - notification_client, - state_store_metrics.clone(), - tracing, - compactor_metrics.clone(), - ) - .await?; + let backup_store = parse_meta_snapshot_storage( + &opts.backup_storage_url, + &opts.backup_storage_directory, + ) + .await?; + let backup_reader = BackupReader::new(backup_store); + let inner = HummockStorage::new( + opts.clone(), + sstable_store, + backup_reader, + hummock_meta_client.clone(), + notification_client, + state_store_metrics.clone(), + tracing, + compactor_metrics.clone(), + ) + .await?; - StateStoreImpl::hummock_v1(inner, storage_metrics) - } + StateStoreImpl::hummock(inner, storage_metrics) } "in_memory" | "in-memory" => { @@ -584,7 +549,7 @@ impl StateStoreImpl { } } -/// This trait is for aligning some common methods of hummock v1 and v2 for external use +/// This trait is for aligning some common methods of `state_store_impl` for external use pub trait HummockTrait { fn sstable_id_manager(&self) -> &SstableIdManagerRef; fn sstable_store(&self) -> SstableStoreRef; @@ -615,28 +580,6 @@ impl HummockTrait for HummockStorage { } } -impl HummockTrait for HummockStorageV1 { - fn sstable_id_manager(&self) -> &SstableIdManagerRef { - self.sstable_id_manager() - } - - fn sstable_store(&self) -> SstableStoreRef { - self.sstable_store() - } - - fn filter_key_extractor_manager(&self) -> &FilterKeyExtractorManagerRef { - self.filter_key_extractor_manager() - } - - fn get_memory_limiter(&self) -> Arc { - self.get_memory_limiter() - } - - fn as_hummock(&self) -> Option<&HummockStorage> { - None - } -} - pub trait AsHummockTrait { fn as_hummock_trait(&self) -> Option<&dyn HummockTrait>; } @@ -647,12 +590,6 @@ impl AsHummockTrait for HummockStorage { } } -impl AsHummockTrait for HummockStorageV1 { - fn as_hummock_trait(&self) -> Option<&dyn HummockTrait> { - Some(self) - } -} - impl AsHummockTrait for MemoryStateStore { fn as_hummock_trait(&self) -> Option<&dyn HummockTrait> { None diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index c9b0ab4e120ee..b8227c72a7882 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -28,7 +28,7 @@ use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{ get_dist_key_in_pk_indices, ColumnDesc, ColumnId, Schema, TableId, TableOption, }; -use risingwave_common::hash::VirtualNode; +use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{self, OwnedRow, Row, RowDeserializer, RowExt}; use risingwave_common::util::ordered::*; use risingwave_common::util::sort_util::OrderType; @@ -391,9 +391,9 @@ impl StorageTable { // distributed tables. assert_eq!(vnode_hint.unwrap_or(DEFAULT_VNODE), DEFAULT_VNODE); - Either::Left(self.vnodes.high_ranges().map(|r| { - let start = Included(VirtualNode::from_index(*r.start()).to_be_bytes().to_vec()); - let end = end_bound_of_prefix(&VirtualNode::from_index(*r.end()).to_be_bytes()); + Either::Left(self.vnodes.vnode_ranges().map(|r| { + let start = Included(r.start().to_be_bytes().to_vec()); + let end = end_bound_of_prefix(&r.end().to_be_bytes()); assert_matches!(end, Excluded(_) | Unbounded); (start, end) })) @@ -403,7 +403,7 @@ impl StorageTable { // If `vnode_hint` is set, we can only access this single vnode. Some(vnode) => Either::Left(std::iter::once(vnode)), // Otherwise, we need to access all vnodes of this table. - None => Either::Right(self.vnodes.iter_ones().map(VirtualNode::from_index)), + None => Either::Right(self.vnodes.iter_vnodes()), }; Either::Right( vnodes.map(|vnode| prefixed_range(encoded_key_range.clone(), &vnode.to_be_bytes())), diff --git a/src/stream/Cargo.toml b/src/stream/Cargo.toml index 2c92d2566066f..baca2796e6198 100644 --- a/src/stream/Cargo.toml +++ b/src/stream/Cargo.toml @@ -21,6 +21,7 @@ async-stream = "0.3" async-trait = "0.1" async_stack_trace = { path = "../utils/async_stack_trace" } bytes = "1" +derivative = "2" dyn-clone = "1" either = "1" enum-as-inner = "0.5" diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 3837a8de9eb0a..cbe47e123fd0c 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -23,7 +23,7 @@ use itertools::{izip, Itertools}; use risingwave_common::array::{Op, StreamChunk, Vis}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{get_dist_key_in_pk_indices, ColumnDesc, TableId, TableOption}; -use risingwave_common::hash::VirtualNode; +use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowDeserializer, RowExt}; use risingwave_common::types::ScalarImpl; use risingwave_common::util::epoch::EpochPair; @@ -812,7 +812,7 @@ impl StateTable { } else { vec![] }; - for vnode in self.vnodes.iter_ones() { + for vnode in self.vnodes.iter_vnodes() { let mut range_begin = vnode.to_be_bytes().to_vec(); let mut range_end = range_begin.clone(); range_begin.extend(&range_begin_suffix); diff --git a/src/stream/src/executor/actor.rs b/src/stream/src/executor/actor.rs index 54c414a5ea458..d11459369bd0f 100644 --- a/src/stream/src/executor/actor.rs +++ b/src/stream/src/executor/actor.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; @@ -22,6 +21,7 @@ use futures::pin_mut; use hytra::TrAdder; use minitrace::prelude::*; use parking_lot::Mutex; +use risingwave_common::error::ErrorSuppressor; use risingwave_common::util::epoch::EpochPair; use risingwave_expr::ExprError; use tokio_stream::StreamExt; @@ -35,15 +35,13 @@ use crate::task::{ActorId, SharedContext}; /// Shared by all operators of an actor. pub struct ActorContext { pub id: ActorId, - fragment_id: u32, - - // TODO: report errors and prompt the user. - pub errors: Mutex>>, + pub fragment_id: u32, last_mem_val: Arc, cur_mem_val: Arc, total_mem_val: Arc>, streaming_metrics: Arc, + pub error_suppressor: Arc>, } pub type ActorContextRef = Arc; @@ -53,11 +51,11 @@ impl ActorContext { Arc::new(Self { id, fragment_id: 0, - errors: Default::default(), cur_mem_val: Arc::new(0.into()), last_mem_val: Arc::new(0.into()), total_mem_val: Arc::new(TrAdder::new()), streaming_metrics: Arc::new(StreamingMetrics::unused()), + error_suppressor: Arc::new(Mutex::new(ErrorSuppressor::new(10))), }) } @@ -66,35 +64,34 @@ impl ActorContext { fragment_id: u32, total_mem_val: Arc>, streaming_metrics: Arc, + unique_user_errors: usize, ) -> ActorContextRef { Arc::new(Self { id, fragment_id, - errors: Default::default(), cur_mem_val: Arc::new(0.into()), last_mem_val: Arc::new(0.into()), total_mem_val, streaming_metrics, + error_suppressor: Arc::new(Mutex::new(ErrorSuppressor::new(unique_user_errors))), }) } pub fn on_compute_error(&self, err: ExprError, identity: &str) { tracing::error!("Compute error: {}, executor: {identity}", err); let executor_name = identity.split(' ').next().unwrap_or("name_not_found"); - self.streaming_metrics - .user_error_count - .with_label_values(&[ - "ExprError", - &err.to_string(), - executor_name, - &self.fragment_id.to_string(), - ]) - .inc(); - self.errors - .lock() - .entry(identity.to_owned()) - .or_default() - .push(err); + let err_str = err.to_string(); + if !self.error_suppressor.lock().suppress_error(&err_str) { + self.streaming_metrics + .user_compute_error_count + .with_label_values(&[ + "ExprError", + &err_str, + executor_name, + &self.fragment_id.to_string(), + ]) + .inc(); + } } pub fn store_mem_usage(&self, val: usize) { diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs index b89c9fa9f6409..1b5f94113d49e 100644 --- a/src/stream/src/executor/dispatch.rs +++ b/src/stream/src/executor/dispatch.rs @@ -305,12 +305,18 @@ impl DispatcherImpl { .map(|&down_id| new_output(context, actor_id, down_id)) .collect::>>()?; + let output_indices = dispatcher + .output_indices + .iter() + .map(|&i| i as usize) + .collect_vec(); + use risingwave_pb::stream_plan::DispatcherType::*; let dispatcher_impl = match dispatcher.get_type()? { Hash => { assert!(!outputs.is_empty()); - let column_indices = dispatcher - .column_indices + let dist_key_indices = dispatcher + .dist_key_indices .iter() .map(|i| *i as usize) .collect(); @@ -320,18 +326,24 @@ impl DispatcherImpl { DispatcherImpl::Hash(HashDataDispatcher::new( outputs, - column_indices, + dist_key_indices, + output_indices, hash_mapping, dispatcher.dispatcher_id, )) } Broadcast => DispatcherImpl::Broadcast(BroadcastDispatcher::new( outputs, + output_indices, dispatcher.dispatcher_id, )), Simple | NoShuffle => { let [output]: [_; 1] = outputs.try_into().unwrap(); - DispatcherImpl::Simple(SimpleDispatcher::new(output, dispatcher.dispatcher_id)) + DispatcherImpl::Simple(SimpleDispatcher::new( + output, + output_indices, + dispatcher.dispatcher_id, + )) } Unspecified => unreachable!(), }; @@ -509,6 +521,7 @@ impl Dispatcher for RoundRobinDataDispatcher { pub struct HashDataDispatcher { outputs: Vec, keys: Vec, + output_indices: Vec, /// Mapping from virtual node to actor id, used for hash data dispatcher to dispatch tasks to /// different downstream actors. hash_mapping: ExpandedActorMapping, @@ -529,12 +542,14 @@ impl HashDataDispatcher { pub fn new( outputs: Vec, keys: Vec, + output_indices: Vec, hash_mapping: ExpandedActorMapping, dispatcher_id: DispatcherId, ) -> Self { Self { outputs, keys, + output_indices, hash_mapping, dispatcher_id, } @@ -592,6 +607,8 @@ impl Dispatcher for HashDataDispatcher { let mut last_vnode_when_update_delete = None; let mut new_ops: Vec = Vec::with_capacity(chunk.capacity()); + // Apply output indices after calculating the vnode. + let chunk = chunk.reorder_columns(&self.output_indices); // TODO: refactor with `Vis`. let (ops, columns, visibility) = chunk.into_inner(); @@ -690,16 +707,19 @@ impl Dispatcher for HashDataDispatcher { #[derive(Debug)] pub struct BroadcastDispatcher { outputs: HashMap, + output_indices: Vec, dispatcher_id: DispatcherId, } impl BroadcastDispatcher { pub fn new( outputs: impl IntoIterator, + output_indices: Vec, dispatcher_id: DispatcherId, ) -> Self { Self { outputs: Self::into_pairs(outputs).collect(), + output_indices, dispatcher_id, } } @@ -718,6 +738,7 @@ impl Dispatcher for BroadcastDispatcher { fn dispatch_data(&mut self, chunk: StreamChunk) -> Self::DataFuture<'_> { async move { + let chunk = chunk.reorder_columns(&self.output_indices); for output in self.outputs.values_mut() { output.send(Message::Chunk(chunk.clone())).await?; } @@ -779,13 +800,19 @@ pub struct SimpleDispatcher { /// Therefore, when dispatching data, we assert that there's exactly one output by /// `Self::output`. output: SmallVec<[BoxedOutput; 2]>, + output_indices: Vec, dispatcher_id: DispatcherId, } impl SimpleDispatcher { - pub fn new(output: BoxedOutput, dispatcher_id: DispatcherId) -> Self { + pub fn new( + output: BoxedOutput, + output_indices: Vec, + dispatcher_id: DispatcherId, + ) -> Self { Self { output: smallvec![output], + output_indices, dispatcher_id, } } @@ -817,6 +844,7 @@ impl Dispatcher for SimpleDispatcher { .exactly_one() .expect("expect exactly one output"); + let chunk = chunk.reorder_columns(&self.output_indices); output.send(Message::Chunk(chunk)).await } } @@ -919,8 +947,13 @@ mod tests { .flat_map(|id| vec![id as ActorId; VirtualNode::COUNT / num_outputs]) .collect_vec(); hash_mapping.resize(VirtualNode::COUNT, num_outputs as u32); - let mut hash_dispatcher = - HashDataDispatcher::new(outputs, key_indices.to_vec(), hash_mapping, 0); + let mut hash_dispatcher = HashDataDispatcher::new( + outputs, + key_indices.to_vec(), + vec![0, 1, 2], + hash_mapping, + 0, + ); let chunk = StreamChunk::from_pretty( " I I I @@ -1147,8 +1180,13 @@ mod tests { .flat_map(|id| vec![id as ActorId; VirtualNode::COUNT / num_outputs]) .collect_vec(); hash_mapping.resize(VirtualNode::COUNT, num_outputs as u32); - let mut hash_dispatcher = - HashDataDispatcher::new(outputs, key_indices.to_vec(), hash_mapping.clone(), 0); + let mut hash_dispatcher = HashDataDispatcher::new( + outputs, + key_indices.to_vec(), + (0..dimension).collect(), + hash_mapping.clone(), + 0, + ); let mut ops = Vec::new(); for idx in 0..cardinality { diff --git a/src/stream/src/executor/dml.rs b/src/stream/src/executor/dml.rs index 9a16b75fbc1d3..181dc247eef57 100644 --- a/src/stream/src/executor/dml.rs +++ b/src/stream/src/executor/dml.rs @@ -12,18 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::future::Either; -use futures::stream::select; +use either::Either; use futures::StreamExt; use futures_async_stream::try_stream; use risingwave_common::catalog::{ColumnDesc, Schema, TableId, TableVersionId}; -use risingwave_connector::source::StreamChunkWithState; use risingwave_source::dml_manager::DmlManagerRef; use super::error::StreamExecutorError; +use super::stream_reader::StreamReaderWithPause; use super::{ - expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, Message, PkIndices, - PkIndicesRef, + expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, Message, Mutation, + PkIndices, PkIndicesRef, }; /// [`DmlExecutor`] accepts both stream data and batch data for data manipulation on a specific @@ -92,31 +91,40 @@ impl DmlExecutor { .dml_manager .register_reader(self.table_id, self.table_version_id, &self.column_descs) .map_err(StreamExecutorError::connector_error)?; - let batch_reader = batch_reader - .stream_reader() - .into_stream() - .map(Either::Right); + let batch_reader = batch_reader.stream_reader().into_stream(); - yield Message::Barrier(barrier); + // Merge the two streams using `StreamReaderWithPause` because when we receive a pause + // barrier, we should stop receiving the data from DML. We poll data from the two streams in + // a round robin way. + let mut stream = StreamReaderWithPause::::new(upstream, batch_reader); - // Stream data from the upstream executor. - let upstream = upstream.map(Either::Left); + // If the first barrier is configuration change, then the DML executor must be newly + // created, and we should start with the paused state. + if barrier.is_update() { + stream.pause_stream(); + } - // Merge the two streams. - let stream = select(upstream, batch_reader); + yield Message::Barrier(barrier); - #[for_await] - for input_msg in stream { - match input_msg { + while let Some(input_msg) = stream.next().await { + match input_msg? { Either::Left(msg) => { - // Stream data. - let msg: Message = msg?; + // Stream messages. + if let Message::Barrier(barrier) = &msg { + // We should handle barrier messages here to pause or resume the data from + // DML. + if let Some(mutation) = barrier.mutation.as_deref() { + match mutation { + Mutation::Pause => stream.pause_stream(), + Mutation::Resume => stream.resume_stream(), + _ => {} + } + } + } yield msg; } Either::Right(chunk) => { // Batch data. - let chunk: StreamChunkWithState = - chunk.map_err(StreamExecutorError::connector_error)?; yield Message::Chunk(chunk.chunk); } } diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index 1f83825522b2f..a52b7f0bed4a9 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -21,7 +21,7 @@ use risingwave_common::array::{Array, ArrayImpl, DataChunk, Op, StreamChunk}; use risingwave_common::bail; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::Schema; -use risingwave_common::hash::VirtualNode; +use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{once, OwnedRow as RowData, Row}; use risingwave_common::types::{DataType, Datum, ScalarImpl, ToDatumRef, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqDebug; @@ -386,11 +386,9 @@ impl DynamicFilterExecutor { let range = (Self::to_row_bound(range.0), Self::to_row_bound(range.1)); // TODO: prefetching for append-only case. - for vnode in self.left_table.vnodes().iter_ones() { - let row_stream = self - .left_table - .iter_with_pk_range(&range, VirtualNode::from_index(vnode)) - .await?; + for vnode in self.left_table.vnodes().iter_vnodes() { + let row_stream = + self.left_table.iter_with_pk_range(&range, vnode).await?; pin_mut!(row_stream); while let Some(res) = row_stream.next().await { let row = res?; diff --git a/src/stream/src/executor/exchange/output.rs b/src/stream/src/executor/exchange/output.rs index c2954d6cd2dd1..737defdae9ada 100644 --- a/src/stream/src/executor/exchange/output.rs +++ b/src/stream/src/executor/exchange/output.rs @@ -17,6 +17,7 @@ use std::fmt::Debug; use anyhow::anyhow; use async_stack_trace::{SpanValue, StackTrace}; use async_trait::async_trait; +use derivative::Derivative; use risingwave_common::util::addr::is_local_address; use tokio::sync::mpsc::error::SendError; @@ -44,22 +45,18 @@ pub trait Output: Debug + Send + Sync + 'static { pub type BoxedOutput = Box; /// `LocalOutput` sends data to a local channel. +#[derive(Derivative)] +#[derivative(Debug)] pub struct LocalOutput { actor_id: ActorId, + #[derivative(Debug = "ignore")] span: SpanValue, + #[derivative(Debug = "ignore")] ch: Sender, } -impl Debug for LocalOutput { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("LocalOutput") - .field("actor_id", &self.actor_id) - .finish() - } -} - impl LocalOutput { pub fn new(actor_id: ActorId, ch: Sender) -> Self { Self { @@ -97,22 +94,18 @@ impl Output for LocalOutput { /// /// [`ExchangeService`]: risingwave_pb::task_service::exchange_service_server::ExchangeService // FIXME: can we just use the same `Output` with local and compacts it in gRPC server? +#[derive(Derivative)] +#[derivative(Debug)] pub struct RemoteOutput { actor_id: ActorId, + #[derivative(Debug = "ignore")] span: SpanValue, + #[derivative(Debug = "ignore")] ch: Sender, } -impl Debug for RemoteOutput { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("RemoteOutput") - .field("actor_id", &self.actor_id) - .finish() - } -} - impl RemoteOutput { pub fn new(actor_id: ActorId, ch: Sender) -> Self { Self { diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index ac7f897a391dc..6331b4a1b3825 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -85,6 +85,7 @@ mod sink; mod sort; mod sort_buffer; pub mod source; +mod stream_reader; pub mod subtask; mod top_n; mod union; diff --git a/src/stream/src/executor/monitor/streaming_stats.rs b/src/stream/src/executor/monitor/streaming_stats.rs index 7c491dae5a670..c01eea929e0ea 100644 --- a/src/stream/src/executor/monitor/streaming_stats.rs +++ b/src/stream/src/executor/monitor/streaming_stats.rs @@ -83,8 +83,8 @@ pub struct StreamingMetrics { pub lru_watermark_step: IntGauge, pub jemalloc_allocated_bytes: IntGauge, - /// User error reporting - pub user_error_count: GenericCounterVec, + /// User compute error reporting + pub user_compute_error_count: GenericCounterVec, } impl StreamingMetrics { @@ -433,9 +433,9 @@ impl StreamingMetrics { ) .unwrap(); - let user_error_count = register_int_counter_vec_with_registry!( - "user_error_count", - "user errors in the system, queryable by tags", + let user_compute_error_count = register_int_counter_vec_with_registry!( + "user_compute_error_count", + "Compute errors in the system, queryable by tags", &["error_type", "error_msg", "executor_name", "fragment_id"], registry, ) @@ -487,7 +487,7 @@ impl StreamingMetrics { lru_runtime_loop_count, lru_watermark_step, jemalloc_allocated_bytes, - user_error_count, + user_compute_error_count, } } diff --git a/src/stream/src/executor/sort.rs b/src/stream/src/executor/sort.rs index 753945f2a6d0f..66e29f16045d7 100644 --- a/src/stream/src/executor/sort.rs +++ b/src/stream/src/executor/sort.rs @@ -20,7 +20,7 @@ use futures_async_stream::try_stream; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; -use risingwave_common::hash::VirtualNode; +use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{self, AscentOwnedRow, OwnedRow, Row, RowExt}; use risingwave_common::types::{ScalarImpl, ToOwnedDatum}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; @@ -259,7 +259,7 @@ impl SortExecutor { curr_vnode_bitmap.to_owned() }; let mut values_per_vnode = Vec::new(); - for owned_vnode in newly_owned_vnodes.iter_ones() { + for owned_vnode in newly_owned_vnodes.iter_vnodes() { let value_iter = self .state_table .iter_with_pk_range( @@ -267,7 +267,7 @@ impl SortExecutor { Bound::::Unbounded, Bound::::Unbounded, ), - VirtualNode::from_index(owned_vnode), + owned_vnode, ) .await?; let value_iter = Box::pin(value_iter); diff --git a/src/stream/src/executor/sort_buffer.rs b/src/stream/src/executor/sort_buffer.rs index e7d96b8ffdb7f..5b24422b009e4 100644 --- a/src/stream/src/executor/sort_buffer.rs +++ b/src/stream/src/executor/sort_buffer.rs @@ -22,7 +22,7 @@ use futures_async_stream::for_await; use gen_iter::GenIter; use risingwave_common::array::{DataChunk, Op, StreamChunk}; use risingwave_common::catalog::Schema; -use risingwave_common::hash::VirtualNode; +use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{self, AscentOwnedRow, OwnedRow, Row, RowExt}; use risingwave_common::types::{ScalarImpl, ToOwnedDatum}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; @@ -77,11 +77,8 @@ impl SortBuffer { Bound::::Unbounded, Bound::::Unbounded, ); - let streams = stream::iter(vnodes.iter_ones()) - .map(|vnode| { - let vnode = VirtualNode::from_index(vnode); - state_table.iter_with_pk_range(&pk_range, vnode) - }) + let streams = stream::iter(vnodes.iter_vnodes()) + .map(|vnode| state_table.iter_with_pk_range(&pk_range, vnode)) .buffer_unordered(10) .try_collect::>() .await? diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 7120e9d5c349b..89320e8ac52a9 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -33,7 +33,7 @@ use super::executor_core::StreamSourceCore; use crate::error::StreamResult; use crate::executor::error::StreamExecutorError; use crate::executor::monitor::StreamingMetrics; -use crate::executor::source::reader::SourceReaderStream; +use crate::executor::stream_reader::StreamReaderWithPause; use crate::executor::*; /// [`FsSourceExecutor`] is a streaming source, fir external file systems /// such as s3. @@ -99,17 +99,21 @@ impl FsSourceExecutor { state, column_ids, source_desc.metrics.clone(), - SourceInfo::new(self.ctx.id, self.stream_source_core.source_id), + SourceInfo::new( + self.ctx.id, + self.stream_source_core.source_id, + self.ctx.fragment_id, + ), ) .await .map_err(StreamExecutorError::connector_error)?; Ok(steam_reader.into_stream()) } - async fn apply_split_change( + async fn apply_split_change( &mut self, source_desc: &FsSourceDesc, - stream: &mut SourceReaderStream, + stream: &mut StreamReaderWithPause, mapping: &HashMap>, ) -> StreamExecutorResult<()> { if let Some(target_splits) = mapping.get(&self.ctx.id).cloned() { @@ -172,10 +176,10 @@ impl FsSourceExecutor { Ok((!no_change_flag).then_some(target_state)) } - async fn replace_stream_reader_with_target_state( + async fn replace_stream_reader_with_target_state( &mut self, source_desc: &FsSourceDesc, - stream: &mut SourceReaderStream, + stream: &mut StreamReaderWithPause, target_state: Vec, ) -> StreamExecutorResult<()> { tracing::info!( @@ -188,7 +192,7 @@ impl FsSourceExecutor { let reader = self .build_stream_source_reader(source_desc, Some(target_state.clone())) .await?; - stream.replace_source_stream(reader); + stream.replace_data_stream(reader); self.stream_source_core.stream_source_splits = target_state .into_iter() @@ -328,10 +332,12 @@ impl FsSourceExecutor { .stack_trace("fs_source_start_reader") .await?; - // Merge the chunks from source and the barriers into a single stream. - let mut stream = SourceReaderStream::new(barrier_receiver, source_chunk_reader); + // Merge the chunks from source and the barriers into a single stream. We prioritize + // barriers over source data chunks here. + let barrier_stream = barrier_to_message_stream(barrier_receiver).boxed(); + let mut stream = StreamReaderWithPause::::new(barrier_stream, source_chunk_reader); if start_with_paused { - stream.pause_source(); + stream.pause_stream(); } yield Message::Barrier(barrier); @@ -345,42 +351,53 @@ impl FsSourceExecutor { while let Some(msg) = stream.next().await { match msg? { // This branch will be preferred. - Either::Left(barrier) => { - last_barrier_time = Instant::now(); - if self_paused { - stream.resume_source(); - self_paused = false; - } - let epoch = barrier.epoch; - - if let Some(ref mutation) = barrier.mutation.as_deref() { - match mutation { - Mutation::SourceChangeSplit(actor_splits) => { - self.apply_split_change(&source_desc, &mut stream, actor_splits) - .await? - } - Mutation::Pause => stream.pause_source(), - Mutation::Resume => stream.resume_source(), - Mutation::Update { actor_splits, .. } => { - self.apply_split_change(&source_desc, &mut stream, actor_splits) + Either::Left(msg) => match &msg { + Message::Barrier(barrier) => { + last_barrier_time = Instant::now(); + if self_paused { + stream.resume_stream(); + self_paused = false; + } + let epoch = barrier.epoch; + + if let Some(ref mutation) = barrier.mutation.as_deref() { + match mutation { + Mutation::SourceChangeSplit(actor_splits) => { + self.apply_split_change(&source_desc, &mut stream, actor_splits) + .await? + } + Mutation::Pause => stream.pause_stream(), + Mutation::Resume => stream.resume_stream(), + Mutation::Update { actor_splits, .. } => { + self.apply_split_change( + &source_desc, + &mut stream, + actor_splits, + ) .await?; + } + _ => {} } - _ => {} } + self.take_snapshot_and_clear_cache(epoch).await?; + + self.metrics + .source_row_per_barrier + .with_label_values(&[ + self.ctx.id.to_string().as_str(), + self.stream_source_core.source_identify.as_ref(), + ]) + .inc_by(metric_row_per_barrier); + metric_row_per_barrier = 0; + + yield msg; } - self.take_snapshot_and_clear_cache(epoch).await?; - - self.metrics - .source_row_per_barrier - .with_label_values(&[ - self.ctx.id.to_string().as_str(), - self.stream_source_core.source_identify.as_ref(), - ]) - .inc_by(metric_row_per_barrier); - metric_row_per_barrier = 0; - - yield Message::Barrier(barrier); - } + _ => { + // For the source executor, the message we receive from this arm should + // always be barrier message. + unreachable!(); + } + }, Either::Right(StreamChunkWithState { chunk, @@ -391,7 +408,7 @@ impl FsSourceExecutor { // we can guarantee the source is not paused since it received stream // chunks. self_paused = true; - stream.pause_source(); + stream.pause_stream(); } // update split offset if let Some(mapping) = split_offset_mapping { diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 1ca31c79696da..7c13505e58545 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -13,13 +13,28 @@ // limitations under the License. pub mod executor_core; +use async_stack_trace::StackTrace; pub use executor_core::StreamSourceCore; mod fs_source_executor; pub use fs_source_executor::*; +use risingwave_common::bail; +pub use state_table_handler::*; pub mod source_executor; -mod reader; pub mod state_table_handler; -pub use state_table_handler::*; +use futures_async_stream::try_stream; +use tokio::sync::mpsc::UnboundedReceiver; + +use crate::executor::error::StreamExecutorError; +use crate::executor::{Barrier, Message}; + +/// Receive barriers from barrier manager with the channel, error on channel close. +#[try_stream(ok = Message, error = StreamExecutorError)] +pub async fn barrier_to_message_stream(mut rx: UnboundedReceiver) { + while let Some(barrier) = rx.recv().stack_trace("receive_barrier").await { + yield Message::Barrier(barrier); + } + bail!("barrier reader closed unexpectedly"); +} diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index d54372bec1b20..1ac2ace79f6cd 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -28,7 +28,7 @@ use tokio::time::Instant; use super::executor_core::StreamSourceCore; use crate::executor::monitor::StreamingMetrics; -use crate::executor::source::reader::SourceReaderStream; +use crate::executor::stream_reader::StreamReaderWithPause; use crate::executor::*; /// A constant to multiply when calculating the maximum time to wait for a barrier. This is due to @@ -100,16 +100,17 @@ impl SourceExecutor { SourceInfo::new( self.ctx.id, self.stream_source_core.as_ref().unwrap().source_id, + self.ctx.fragment_id, ), ) .await .map_err(StreamExecutorError::connector_error) } - async fn apply_split_change( + async fn apply_split_change( &mut self, source_desc: &SourceDesc, - stream: &mut SourceReaderStream, + stream: &mut StreamReaderWithPause, mapping: &HashMap>, ) -> StreamExecutorResult<()> { if let Some(target_splits) = mapping.get(&self.ctx.id).cloned() { @@ -163,10 +164,10 @@ impl SourceExecutor { Ok((!no_change_flag).then_some(target_state)) } - async fn replace_stream_reader_with_target_state( + async fn replace_stream_reader_with_target_state( &mut self, source_desc: &SourceDesc, - stream: &mut SourceReaderStream, + stream: &mut StreamReaderWithPause, target_state: Vec, ) -> StreamExecutorResult<()> { tracing::info!( @@ -179,7 +180,7 @@ impl SourceExecutor { let reader = self .build_stream_source_reader(source_desc, Some(target_state.clone())) .await?; - stream.replace_source_stream(reader); + stream.replace_data_stream(reader); self.stream_source_core .as_mut() @@ -289,13 +290,15 @@ impl SourceExecutor { .stack_trace("source_build_reader") .await?; - // Merge the chunks from source and the barriers into a single stream. - let mut stream = SourceReaderStream::new(barrier_receiver, source_chunk_reader); + // Merge the chunks from source and the barriers into a single stream. We prioritize + // barriers over source data chunks here. + let barrier_stream = barrier_to_message_stream(barrier_receiver).boxed(); + let mut stream = StreamReaderWithPause::::new(barrier_stream, source_chunk_reader); // If the first barrier is configuration change, then the source executor must be newly // created, and we should start with the paused state. if barrier.is_update() { - stream.pause_source(); + stream.pause_stream(); } yield Message::Barrier(barrier); @@ -310,47 +313,58 @@ impl SourceExecutor { while let Some(msg) = stream.next().await { match msg? { // This branch will be preferred. - Either::Left(barrier) => { - last_barrier_time = Instant::now(); - if self_paused { - stream.resume_source(); - self_paused = false; - } - let epoch = barrier.epoch; - - if let Some(ref mutation) = barrier.mutation.as_deref() { - match mutation { - Mutation::SourceChangeSplit(actor_splits) => { - self.apply_split_change(&source_desc, &mut stream, actor_splits) - .await? - } - Mutation::Pause => stream.pause_source(), - Mutation::Resume => stream.resume_source(), - Mutation::Update { actor_splits, .. } => { - self.apply_split_change(&source_desc, &mut stream, actor_splits) + Either::Left(msg) => match &msg { + Message::Barrier(barrier) => { + last_barrier_time = Instant::now(); + if self_paused { + stream.resume_stream(); + self_paused = false; + } + let epoch = barrier.epoch; + + if let Some(ref mutation) = barrier.mutation.as_deref() { + match mutation { + Mutation::SourceChangeSplit(actor_splits) => { + self.apply_split_change(&source_desc, &mut stream, actor_splits) + .await? + } + Mutation::Pause => stream.pause_stream(), + Mutation::Resume => stream.resume_stream(), + Mutation::Update { actor_splits, .. } => { + self.apply_split_change( + &source_desc, + &mut stream, + actor_splits, + ) .await?; + } + _ => {} } - _ => {} } - } - self.take_snapshot_and_clear_cache(epoch).await?; + self.take_snapshot_and_clear_cache(epoch).await?; - self.metrics - .source_row_per_barrier - .with_label_values(&[ - self.ctx.id.to_string().as_str(), - self.stream_source_core - .as_ref() - .unwrap() - .source_identify - .as_ref(), - ]) - .inc_by(metric_row_per_barrier); - metric_row_per_barrier = 0; + self.metrics + .source_row_per_barrier + .with_label_values(&[ + self.ctx.id.to_string().as_str(), + self.stream_source_core + .as_ref() + .unwrap() + .source_identify + .as_ref(), + ]) + .inc_by(metric_row_per_barrier); + metric_row_per_barrier = 0; - yield Message::Barrier(barrier); - } + yield msg; + } + _ => { + // For the source executor, the message we receive from this arm should + // always be barrier message. + unreachable!(); + } + }, Either::Right(StreamChunkWithState { chunk, @@ -361,7 +375,7 @@ impl SourceExecutor { // we can guarantee the source is not paused since it received stream // chunks. self_paused = true; - stream.pause_source(); + stream.pause_stream(); } if let Some(mapping) = split_offset_mapping { let state: HashMap<_, _> = mapping diff --git a/src/stream/src/executor/source/reader.rs b/src/stream/src/executor/stream_reader.rs similarity index 56% rename from src/stream/src/executor/source/reader.rs rename to src/stream/src/executor/stream_reader.rs index b801b853eb0eb..aef2870a67e12 100644 --- a/src/stream/src/executor/source/reader.rs +++ b/src/stream/src/executor/stream_reader.rs @@ -20,37 +20,38 @@ use either::Either; use futures::stream::{select_with_strategy, BoxStream, PollNext, SelectWithStrategy}; use futures::{Stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; -use risingwave_common::bail; use risingwave_connector::source::{BoxSourceWithStateStream, StreamChunkWithState}; -use tokio::sync::mpsc::UnboundedReceiver; use crate::executor::error::{StreamExecutorError, StreamExecutorResult}; -use crate::executor::Barrier; - -type SourceReaderMessage = StreamExecutorResult>; -type SourceReaderArm = BoxStream<'static, SourceReaderMessage>; -type SourceReaderStreamInner = - SelectWithStrategy PollNext, ()>; - -pub(super) struct SourceReaderStream { - inner: SourceReaderStreamInner, +use crate::executor::Message; + +type ExecutorMessageStream = BoxStream<'static, StreamExecutorResult>; +type StreamReaderData = StreamExecutorResult>; +type ReaderArm = BoxStream<'static, StreamReaderData>; +type StreamReaderWithPauseInner = + SelectWithStrategy PollNext, PollNext>; + +/// [`StreamReaderWithPause`] merges two streams, with one receiving barriers (and maybe other types +/// of messages) and the other receiving data only (no barrier). The merged stream can be paused +/// (`StreamReaderWithPause::pause_stream`) and resumed (`StreamReaderWithPause::resume_stream`). +/// A paused stream will not receive any data from either original stream until a barrier arrives +/// and the stream is resumed. +/// +/// ## Priority +/// +/// If `BIASED` is `true`, the left-hand stream (the one receiving barriers) will get a higher +/// priority over the right-hand one. Otherwise, the two streams will be polled in a round robin +/// fashion. +pub(super) struct StreamReaderWithPause { + inner: StreamReaderWithPauseInner, /// Whether the source stream is paused. paused: bool, } -impl SourceReaderStream { - /// Receive barriers from barrier manager with the channel, error on channel close. - #[try_stream(ok = Barrier, error = StreamExecutorError)] - async fn barrier_receiver(mut rx: UnboundedReceiver) { - while let Some(barrier) = rx.recv().stack_trace("source_recv_barrier").await { - yield barrier; - } - bail!("barrier reader closed unexpectedly"); - } - - /// Receive chunks and states from the source reader, hang up on error. +impl StreamReaderWithPause { + /// Receive chunks and states from the reader. Hang up on error. #[try_stream(ok = StreamChunkWithState, error = StreamExecutorError)] - async fn source_stream(stream: BoxSourceWithStateStream) { + async fn data_stream(stream: BoxSourceWithStateStream) { // TODO: support stack trace for Stream #[for_await] for chunk in stream { @@ -64,38 +65,33 @@ impl SourceReaderStream { } } - /// Convert this reader to a stream. + /// Construct a `StreamReaderWithPause` with one stream receiving barrier messages (and maybe + /// other types of messages) and the other receiving data only (no barrier). pub fn new( - barrier_receiver: UnboundedReceiver, - source_stream: BoxSourceWithStateStream, + message_stream: ExecutorMessageStream, + data_stream: BoxSourceWithStateStream, ) -> Self { + let message_stream_arm = message_stream.map_ok(Either::Left).boxed(); + let data_stream_arm = Self::data_stream(data_stream).map_ok(Either::Right).boxed(); + let inner = Self::new_inner(message_stream_arm, data_stream_arm); Self { - inner: Self::new_inner( - Self::barrier_receiver(barrier_receiver) - .map_ok(Either::Left) - .boxed(), - Self::source_stream(source_stream) - .map_ok(Either::Right) - .boxed(), - ), + inner, paused: false, } } - fn new_inner( - barrier_receiver_arm: SourceReaderArm, - source_stream_arm: SourceReaderArm, - ) -> SourceReaderStreamInner { - select_with_strategy( - barrier_receiver_arm, - source_stream_arm, - // We prefer barrier on the left hand side over source chunks. - |_: &mut ()| PollNext::Left, - ) + fn new_inner(message_stream: ReaderArm, data_stream: ReaderArm) -> StreamReaderWithPauseInner { + let strategy = if BIASED { + |_: &mut PollNext| PollNext::Left + } else { + // The poll strategy is not biased: we poll the two streams in a round robin way. + |last: &mut PollNext| last.toggle() + }; + select_with_strategy(message_stream, data_stream, strategy) } - /// Replace the source stream with a new one for given `stream`. Used for split change. - pub fn replace_source_stream(&mut self, source_stream: BoxSourceWithStateStream) { + /// Replace the data stream with a new one for given `stream`. Used for split change. + pub fn replace_data_stream(&mut self, data_stream: BoxSourceWithStateStream) { // Take the barrier receiver arm. let barrier_receiver_arm = std::mem::replace( self.inner.get_mut().0, @@ -106,35 +102,38 @@ impl SourceReaderStream { // to ensure the internal state of the `SelectWithStrategy` is reset. (#6300) self.inner = Self::new_inner( barrier_receiver_arm, - Self::source_stream(source_stream) - .map_ok(Either::Right) - .boxed(), + Self::data_stream(data_stream).map_ok(Either::Right).boxed(), ); } - /// Pause the source stream. - pub fn pause_source(&mut self) { + /// Pause the data stream. + pub fn pause_stream(&mut self) { assert!(!self.paused, "already paused"); self.paused = true; } - /// Resume the source stream, panic if the source is not paused before. - pub fn resume_source(&mut self) { + /// Resume the data stream. Panic if the data stream is not paused. + pub fn resume_stream(&mut self) { assert!(self.paused, "not paused"); self.paused = false; } } -impl Stream for SourceReaderStream { - type Item = SourceReaderMessage; +impl Stream for StreamReaderWithPause { + type Item = StreamReaderData; fn poll_next( mut self: Pin<&mut Self>, ctx: &mut std::task::Context<'_>, ) -> Poll> { if self.paused { + // Note: It is safe here to poll the left arm even if it contains streaming messages + // other than barriers: after the upstream executor sends a `Mutation::Pause`, there + // should be no more message until a `Mutation::Update` and a 'Mutation::Resume`. self.inner.get_mut().0.poll_next_unpin(ctx) } else { + // TODO: We may need to prioritize the data stream (right-hand stream) after resuming + // from the paused state. self.inner.poll_next_unpin(ctx) } } @@ -149,6 +148,7 @@ mod tests { use tokio::sync::mpsc; use super::*; + use crate::executor::{barrier_to_message_stream, Barrier}; #[tokio::test] async fn test_pause_and_resume() { @@ -157,7 +157,8 @@ mod tests { let table_dml_handle = TableDmlHandle::new(vec![]); let source_stream = table_dml_handle.stream_reader().into_stream(); - let stream = SourceReaderStream::new(barrier_rx, source_stream); + let barrier_stream = barrier_to_message_stream(barrier_rx).boxed(); + let stream = StreamReaderWithPause::::new(barrier_stream, source_stream); pin_mut!(stream); macro_rules! next { @@ -181,7 +182,7 @@ mod tests { assert_matches!(next!().unwrap(), Either::Left(_)); // Pause the stream. - stream.pause_source(); + stream.pause_stream(); // Write a barrier. barrier_tx.send(Barrier::new_test_barrier(2)).unwrap(); @@ -197,7 +198,7 @@ mod tests { assert!(next!().is_none()); // Resume the stream. - stream.resume_source(); + stream.resume_stream(); // Then we can receive the chunk sent when the stream is paused. assert_matches!(next!().unwrap(), Either::Right(_)); } diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index 9ecd56330b392..5c9be4aba4b9d 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -18,7 +18,7 @@ use futures::future::join_all; use futures::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::hash::VirtualNode; +use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::{bail, row}; @@ -212,8 +212,8 @@ impl WatermarkFilterExecutor { last_checkpoint_watermark = current_watermark.clone(); // Persist the watermark when checkpoint arrives. let vnodes = table.get_vnodes(); - for vnode in vnodes.iter_ones() { - let pk = Some(ScalarImpl::Int16(vnode as _)); + for vnode in vnodes.iter_vnodes() { + let pk = Some(ScalarImpl::Int16(vnode.to_scalar())); let row = [pk, Some(current_watermark.clone())]; // FIXME(yuhao): use upsert. table.insert(row); diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index d3e5ffd11e0b2..54e94f832c421 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -607,6 +607,7 @@ impl LocalStreamManagerCore { actor.fragment_id, self.total_mem_val.clone(), self.streaming_metrics.clone(), + self.config.unique_user_stream_errors, ); let vnode_bitmap = actor .vnode_bitmap diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index ed6bd69787ea3..5bfaf1f3177e1 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -87,8 +87,7 @@ pub fn start_delete_range(opts: CompactionTestOpts) -> Pin anyhow::Result<()> { - let mut config = load_config(&opts.config_path, NO_OVERRIDE); - config.storage.enable_state_store_v1 = false; + let config = load_config(&opts.config_path, NO_OVERRIDE); let compaction_config = CompactionConfigBuilder::new().build(); compaction_test(compaction_config, config, &opts.state_store, 1000000, 800).await } @@ -601,20 +600,14 @@ fn run_compactor_thread( #[cfg(test)] mod tests { - use risingwave_common::config::{RwConfig, StorageConfig}; + use risingwave_common::config::RwConfig; use risingwave_meta::hummock::compaction::compaction_config::CompactionConfigBuilder; use super::compaction_test; #[tokio::test(flavor = "multi_thread", worker_threads = 3)] async fn test_small_data() { - let config = RwConfig { - storage: StorageConfig { - enable_state_store_v1: false, - ..Default::default() - }, - ..Default::default() - }; + let config = RwConfig::default(); let mut compaction_config = CompactionConfigBuilder::new().build(); compaction_config.max_sub_compaction = 1; compaction_config.level0_tier_compact_file_number = 2; diff --git a/src/udf/Cargo.toml b/src/udf/Cargo.toml index 951a93ced41db..36820820a618f 100644 --- a/src/udf/Cargo.toml +++ b/src/udf/Cargo.toml @@ -11,9 +11,9 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] -arrow-array = "31" -arrow-flight = "31" -arrow-schema = "31" +arrow-array = "33" +arrow-flight = "33" +arrow-schema = "33" futures-util = "0.3.25" thiserror = "1" tokio = { version = "0.2", package = "madsim-tokio", features = ["rt", "macros"] } diff --git a/src/udf/README.md b/src/udf/README.md index ee41f396548b1..6364e8513a059 100644 --- a/src/udf/README.md +++ b/src/udf/README.md @@ -7,7 +7,7 @@ ```sh pip3 install pyarrow # run server -python3 arrow_flight.py +python3 python/example.py # run client (test client for the arrow flight UDF client-server protocol) cargo run --example client ``` diff --git a/src/udf/arrow_flight.py b/src/udf/arrow_flight.py deleted file mode 100644 index f464a12332adb..0000000000000 --- a/src/udf/arrow_flight.py +++ /dev/null @@ -1,55 +0,0 @@ -import pathlib - -import pyarrow as pa -import pyarrow.flight -import pyarrow.parquet - - -class FlightServer(pa.flight.FlightServerBase): - """ - Reference: https://arrow.apache.org/cookbook/py/flight.html#simple-parquet-storage-service-with-arrow-flight - """ - - def __init__(self, location="grpc://0.0.0.0:8815", **kwargs): - super(FlightServer, self).__init__(location, **kwargs) - self._location = location - self._functions = {} - - def get_flight_info(self, context, descriptor): - """Return a FlightInfo.""" - return pa.flight.FlightInfo(schema=pa.schema([ - ('c', pa.int32()), - ]), descriptor=descriptor, endpoints=[], total_records=0, total_bytes=0) - - def add_function(self, name: str, func): - """Add a function to the server.""" - self._functions[name] = func - - def do_exchange(self, context, descriptor, reader, writer): - """Run a simple echo server.""" - func = self._functions[descriptor.path[0].decode('utf-8')] - schema = pa.schema([ - ('c', pa.int32()), - ]) - writer.begin(schema) - for chunk in reader: - print(pa.Table.from_batches([chunk.data])) - result = self.call_func(func, chunk.data) - writer.write_table(result) - - def call_func(self, func, batch: pa.RecordBatch) -> pa.Table: - data = pa.array([func(batch[0][i].as_py(), batch[1][i].as_py()) - for i in range(len(batch[0]))]) - return pa.Table.from_arrays([data], names=['c']) - - -def gcd(x: int, y: int) -> int: - while y != 0: - (x, y) = (y, x % y) - return x - - -if __name__ == '__main__': - server = FlightServer() - server.add_function("gcd", gcd) - server.serve() diff --git a/src/udf/examples/client.rs b/src/udf/examples/client.rs index 034d3816eab48..415e00b3151ec 100644 --- a/src/udf/examples/client.rs +++ b/src/udf/examples/client.rs @@ -24,28 +24,51 @@ async fn main() { let client = ArrowFlightUdfClient::connect(addr).await.unwrap(); // build `RecordBatch` to send (equivalent to our `DataChunk`) - let array1 = Int32Array::from_iter(vec![1, 6, 10]); - let array2 = Int32Array::from_iter(vec![3, 4, 15]); - let input_schema = Schema::new(vec![ + let array1 = Arc::new(Int32Array::from_iter(vec![1, 6, 10])); + let array2 = Arc::new(Int32Array::from_iter(vec![3, 4, 15])); + let array3 = Arc::new(Int32Array::from_iter(vec![6, 8, 3])); + let input2_schema = Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), ]); - let output_schema = Schema::new(vec![Field::new("c", DataType::Int32, true)]); + let input3_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let output_schema = Schema::new(vec![Field::new("x", DataType::Int32, true)]); // check function - let id = client - .check("gcd", &input_schema, &output_schema) + let gcd2 = client + .check("gcd", &input2_schema, &output_schema) .await .unwrap(); + let gcd3 = client + .check("gcd", &input3_schema, &output_schema) + .await + .unwrap(); + + let input2 = RecordBatch::try_new( + Arc::new(input2_schema), + vec![array1.clone(), array2.clone()], + ) + .unwrap(); + + let output = client + .call(&gcd2, input2) + .await + .expect("failed to call function"); + + println!("{:?}", output); - let input = RecordBatch::try_new( - Arc::new(input_schema), - vec![Arc::new(array1), Arc::new(array2)], + let input3 = RecordBatch::try_new( + Arc::new(input3_schema), + vec![array1.clone(), array2.clone(), array3.clone()], ) .unwrap(); let output = client - .call(&id, input) + .call(&gcd3, input3) .await .expect("failed to call function"); diff --git a/src/udf/python/example.py b/src/udf/python/example.py new file mode 100644 index 0000000000000..1b615728fe015 --- /dev/null +++ b/src/udf/python/example.py @@ -0,0 +1,27 @@ +from risingwave.udf import udf, UdfServer +import random + + +@udf(input_types=[], result_type='INT') +def random_int() -> int: + return random.randint(0, 100) + + +@udf(input_types=['INT', 'INT'], result_type='INT') +def gcd(x: int, y: int) -> int: + while y != 0: + (x, y) = (y, x % y) + return x + + +@udf(name='gcd', input_types=['INT', 'INT', 'INT'], result_type='INT') +def gcd3(x: int, y: int, z: int) -> int: + return gcd(gcd(x, y), z) + + +if __name__ == '__main__': + server = UdfServer() + server.add_function(random_int) + server.add_function(gcd) + server.add_function(gcd3) + server.serve() diff --git a/src/udf/python/risingwave/__init__.py b/src/udf/python/risingwave/__init__.py new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/udf/python/risingwave/udf.py b/src/udf/python/risingwave/udf.py new file mode 100644 index 0000000000000..bdce30a56387a --- /dev/null +++ b/src/udf/python/risingwave/udf.py @@ -0,0 +1,168 @@ +from typing import * +import pyarrow as pa +import pyarrow.flight +import pyarrow.parquet + + +class UserDefinedFunction: + """ + Base interface for user-defined function. + """ + _name: str + _input_types: List[pa.DataType] + _result_type: pa.DataType + + def full_name(self) -> str: + """ + A unique name for the function. Composed by function name and input types. + Example: "gcd/int32,int32" + """ + return self._name + '/' + ','.join([str(t) for t in self._input_types]) + + def result_schema(self) -> pa.Schema: + """ + Returns the schema of the result table. + """ + return pa.schema([('', self._result_type)]) + + def eval_batch(self, batch: pa.RecordBatch) -> pa.RecordBatch: + """ + Apply the function on a batch of inputs. + """ + pass + + +class ScalarFunction(UserDefinedFunction): + """ + Base interface for user-defined scalar function. A user-defined scalar functions maps zero, one, + or multiple scalar values to a new scalar value. + """ + + def eval(self, *args): + """ + Method which defines the logic of the scalar function. + """ + pass + + def eval_batch(self, batch: pa.RecordBatch) -> pa.RecordBatch: + result = pa.array([self.eval(*[col[i].as_py() for col in batch]) + for i in range(batch.num_rows)], + type=self._result_type) + return pa.RecordBatch.from_arrays([result], schema=self.result_schema()) + + +class UserDefinedFunctionWrapper(ScalarFunction): + """ + Base Wrapper for Python user-defined function. + """ + _func: Callable + + def __init__(self, func, input_types, result_type, name=None): + self._func = func + self._input_types = [_to_data_type(t) for t in input_types] + self._result_type = _to_data_type(result_type) + self._name = name or ( + func.__name__ if hasattr(func, '__name__') else func.__class__.__name__) + + def __call__(self, *args): + return self._func(*args) + + def eval(self, *args): + return self._func(*args) + + +def _create_udf(f, input_types, result_type, name): + return UserDefinedFunctionWrapper( + f, input_types, result_type, name) + + +def udf(input_types: Union[List[Union[str, pa.DataType]], Union[str, pa.DataType]], + result_type: Union[str, pa.DataType], + name: Optional[str] = None,) -> Union[Callable, UserDefinedFunction]: + """ + Annotation for creating a user-defined function. + """ + + return lambda f: _create_udf(f, input_types, result_type, name) + + +class UdfServer(pa.flight.FlightServerBase): + """ + UDF server based on Apache Arrow Flight protocol. + Reference: https://arrow.apache.org/cookbook/py/flight.html#simple-parquet-storage-service-with-arrow-flight + """ + _functions: Dict[str, UserDefinedFunction] + + def __init__(self, location="grpc://0.0.0.0:8815", **kwargs): + super(UdfServer, self).__init__(location, **kwargs) + self._functions = {} + + def get_flight_info(self, context, descriptor): + """Return the result schema of a function.""" + udf = self._functions[descriptor.path[0].decode('utf-8')] + return pa.flight.FlightInfo(schema=udf.result_schema(), descriptor=descriptor, endpoints=[], total_records=0, total_bytes=0) + + def add_function(self, udf: UserDefinedFunction): + """Add a function to the server.""" + name = udf.full_name() + if name in self._functions: + raise ValueError('Function already exists: ' + name) + print('added function:', name) + self._functions[name] = udf + + def do_exchange(self, context, descriptor, reader, writer): + """Call a function from the client.""" + udf = self._functions[descriptor.path[0].decode('utf-8')] + writer.begin(udf.result_schema()) + for chunk in reader: + # print(pa.Table.from_batches([chunk.data])) + result = udf.eval_batch(chunk.data) + writer.write_batch(result) + + def serve(self): + """Start the server.""" + super(UdfServer, self).serve() + + +def _to_data_type(t: Union[str, pa.DataType]) -> pa.DataType: + """ + Convert a string or pyarrow.DataType to pyarrow.DataType. + """ + if isinstance(t, str): + return _string_to_data_type(t) + else: + return t + + +def _string_to_data_type(type_str: str): + match type_str: + case 'BOOLEAN': + return pa.bool_() + case 'TINYINT': + return pa.int8() + case 'SMALLINT': + return pa.int16() + case 'INT' | 'INTEGER': + return pa.int32() + case 'BIGINT': + return pa.int64() + case 'FLOAT' | 'REAL': + return pa.float32() + case 'DOUBLE': + return pa.float64() + case 'DECIMAL': + return pa.decimal128(38) + case 'DATE': + return pa.date32() + case 'DATETIME': + return pa.timestamp('ms') + case 'TIME': + return pa.time32('ms') + case 'TIMESTAMP': + return pa.timestamp('us') + case 'CHAR' | 'VARCHAR': + return pa.string() + case 'BINARY' | 'VARBINARY': + return pa.binary() + case _: + raise ValueError(f'Unsupported type: {type_str}') diff --git a/src/udf/src/lib.rs b/src/udf/src/lib.rs index 6d8a2bb8f3250..4a4d48ab7ef63 100644 --- a/src/udf/src/lib.rs +++ b/src/udf/src/lib.rs @@ -38,11 +38,15 @@ impl ArrowFlightUdfClient { /// Check if the function is available and return the function ID. pub async fn check(&self, name: &str, args: &Schema, returns: &Schema) -> Result { - let mut path = vec![name.to_string()]; - for arg in &args.fields { - path.push(format!("{}", arg.data_type())); + // path = name/[args,]* + let mut path = name.to_string() + "/"; + for (i, arg) in args.fields.iter().enumerate() { + if i != 0 { + path += ","; + } + path += &arg.data_type().to_string().to_lowercase(); } - let descriptor = FlightDescriptor::new_path(path.clone()); + let descriptor = FlightDescriptor::new_path(vec![path.clone()]); let response = self.client.clone().get_flight_info(descriptor).await?; @@ -58,7 +62,7 @@ impl ArrowFlightUdfClient { actual: format!("{:?}", actual_types), }); } - Ok(FunctionId(path)) + Ok(FunctionId(vec![path])) } /// Call a function. diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 853fdcc9cb912..14680804350d5 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -92,7 +92,7 @@ stable_deref_trait = { version = "1" } strum = { version = "0.24", features = ["derive"] } time = { version = "0.3", features = ["formatting", "local-offset", "macros", "parsing"] } tokio = { version = "1", features = ["fs", "io-std", "io-util", "macros", "net", "parking_lot", "process", "rt-multi-thread", "signal", "stats", "sync", "time", "tracing"] } -tokio-stream = { version = "0.1", features = ["net"] } +tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "0c25710", features = ["net"] } tokio-util = { version = "0.7", features = ["codec", "io"] } tonic = { version = "0.8", features = ["gzip", "tls-webpki-roots"] } tower = { version = "0.4", features = ["balance", "buffer", "filter", "limit", "load-shed", "retry", "timeout", "util"] } @@ -187,7 +187,7 @@ strum = { version = "0.24", features = ["derive"] } syn = { version = "1", features = ["extra-traits", "full", "visit", "visit-mut"] } time = { version = "0.3", features = ["formatting", "local-offset", "macros", "parsing"] } tokio = { version = "1", features = ["fs", "io-std", "io-util", "macros", "net", "parking_lot", "process", "rt-multi-thread", "signal", "stats", "sync", "time", "tracing"] } -tokio-stream = { version = "0.1", features = ["net"] } +tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "0c25710", features = ["net"] } tokio-util = { version = "0.7", features = ["codec", "io"] } tonic = { version = "0.8", features = ["gzip", "tls-webpki-roots"] } tonic-build = { version = "0.8" }