diff --git a/Cargo.lock b/Cargo.lock index 110111363de05..8d83cb3fa3397 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2805,6 +2805,18 @@ dependencies = [ "tempfile", ] +[[package]] +name = "nix" +version = "0.24.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f17df307904acd05aa8e32e97bb20f2a0df1728bbc2d771ae8f9a90463441e9" +dependencies = [ + "bitflags", + "cfg-if 1.0.0", + "libc", + "memoffset 0.6.5", +] + [[package]] name = "nom" version = "2.2.1" @@ -3885,6 +3897,7 @@ dependencies = [ name = "risingwave_cmd" version = "0.1.9" dependencies = [ + "anyhow", "clap 3.1.18", "log", "madsim", @@ -4017,6 +4030,7 @@ dependencies = [ "madsim", "madsim-tokio", "madsim-tonic", + "maplit", "memcomparable", "num-traits", "paste", @@ -4026,6 +4040,7 @@ dependencies = [ "rdkafka", "risingwave_batch", "risingwave_common", + "risingwave_connector", "risingwave_pb", "risingwave_rpc_client", "risingwave_source", @@ -4117,7 +4132,10 @@ dependencies = [ "anyhow", "bytes", "clap 3.1.18", + "futures", + "madsim-tokio", "risingwave_common", + "risingwave_frontend", "risingwave_hummock_sdk", "risingwave_pb", "risingwave_rpc_client", @@ -4516,6 +4534,7 @@ dependencies = [ "madsim-tonic", "memcomparable", "moka", + "nix", "num-integer", "num-traits", "parking_lot", @@ -4574,6 +4593,7 @@ dependencies = [ "madsim", "madsim-tokio", "madsim-tonic", + "maplit", "memcomparable", "num-traits", "parking_lot", @@ -4585,6 +4605,7 @@ dependencies = [ "risingwave_common", "risingwave_connector", "risingwave_expr", + "risingwave_hummock_sdk", "risingwave_pb", "risingwave_rpc_client", "risingwave_source", diff --git a/Makefile.toml b/Makefile.toml index 5dcb27bfc641b..bd37dca322e2f 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -297,6 +297,25 @@ set -ex RUST_BACKTRACE=1 RW_NODE=playground cargo run --bin risingwave --profile "${RISINGWAVE_BUILD_PROFILE}" ''' +[tasks.ctl] +category = "RiseDev - Start" +description = "Start RiseCtl" +script = ''' +#!@shell + +RC_ENV_FILE="${PREFIX_CONFIG}/risectl-env" + +if [ ! -f "${RC_ENV_FILE}" ]; then + echo "risectl-env file not found. Did you start cluster using $(tput setaf 4)\`./risedev d\`$(tput sgr0)?" + exit 1 +fi + +source "${RC_ENV_FILE}" + +cargo run --bin risectl --profile "${RISINGWAVE_BUILD_PROFILE}" -- "$@" +test $? -eq 0 || exit 1 +''' + [tasks.d] alias = "dev" diff --git a/ci/scripts/common.env.sh b/ci/scripts/common.env.sh index f967c57b68eba..dc2ce7f117e36 100644 --- a/ci/scripts/common.env.sh +++ b/ci/scripts/common.env.sh @@ -2,3 +2,4 @@ export CARGO_TERM_COLOR=always export RUSTFLAGS="-D warnings" export PROTOC_NO_VENDOR=true export CARGO_HOME=/risingwave/.cargo +export RISINGWAVE_CI=true diff --git a/docs/developer-guide.md b/docs/developer-guide.md index 444d050afa9eb..2add01f9c8234 100644 --- a/docs/developer-guide.md +++ b/docs/developer-guide.md @@ -22,6 +22,7 @@ To report bugs, create a [GitHub issue](https://github.com/singularity-data/risi - [Dashboard v1](#dashboard-v1) - [Dashboard v2](#dashboard-v2) - [Observability components](#observability-components) + - [Cluster Control](#cluster-control) - [Monitoring](#monitoring) - [Tracing](#tracing) - [Dashboard](#dashboard) @@ -199,6 +200,22 @@ The development instructions for dashboard v2 are available [here](../dashboard/ RiseDev supports several observability components. +### Cluster Control + +`risectl` is the tool for providing internal access to the RisingWave cluster. See + +``` +cargo run --bin risectl -- --help +``` + +... or + +``` +./risingwave risectl --help +``` + +for more information. + ### Monitoring Uncomment `grafana` and `prometheus` lines in `risedev.yml` to enable Grafana and Prometheus services. @@ -339,17 +356,6 @@ workspace. And use [cargo-sort](https://crates.io/crates/cargo-sort) to ensure all deps are get sorted. -## Check in PRs from forks - -```shell -gh pr checkout -git checkout -b forks/ -git push origin HEAD -u -``` - -After that, CI checks will begin on branches of RisingWave's main repo, -and the status will be automatically updated to PRs from forks. - ## Submit PRs Instructions about submitting PRs are included in the [contribution guidelines](../CONTRIBUTING.md). diff --git a/e2e_test/batch/aggregate/avg.slt b/e2e_test/batch/aggregate/avg.slt.part similarity index 100% rename from e2e_test/batch/aggregate/avg.slt rename to e2e_test/batch/aggregate/avg.slt.part diff --git a/e2e_test/batch/aggregate/count.slt b/e2e_test/batch/aggregate/count.slt.part similarity index 100% rename from e2e_test/batch/aggregate/count.slt rename to e2e_test/batch/aggregate/count.slt.part diff --git a/e2e_test/batch/aggregate/distinct.slt b/e2e_test/batch/aggregate/distinct.slt.part similarity index 100% rename from e2e_test/batch/aggregate/distinct.slt rename to e2e_test/batch/aggregate/distinct.slt.part diff --git a/e2e_test/batch/aggregate/min_max.slt b/e2e_test/batch/aggregate/min_max.slt.part similarity index 100% rename from e2e_test/batch/aggregate/min_max.slt rename to e2e_test/batch/aggregate/min_max.slt.part diff --git a/e2e_test/batch/aggregate/single_value.slt b/e2e_test/batch/aggregate/single_value.slt.part similarity index 100% rename from e2e_test/batch/aggregate/single_value.slt rename to e2e_test/batch/aggregate/single_value.slt.part diff --git a/e2e_test/batch/aggregate/sum.slt b/e2e_test/batch/aggregate/sum.slt.part similarity index 100% rename from e2e_test/batch/aggregate/sum.slt rename to e2e_test/batch/aggregate/sum.slt.part diff --git a/e2e_test/batch/aggregate/timestamp.slt b/e2e_test/batch/aggregate/timestamp.slt.part similarity index 100% rename from e2e_test/batch/aggregate/timestamp.slt rename to e2e_test/batch/aggregate/timestamp.slt.part diff --git a/e2e_test/batch/distribution_mode.slt b/e2e_test/batch/distribution_mode.slt index 3d19b10b4dd74..0b7438688c25b 100644 --- a/e2e_test/batch/distribution_mode.slt +++ b/e2e_test/batch/distribution_mode.slt @@ -6,3 +6,5 @@ SET QUERY_MODE TO distributed; include ./basic/*.slt.part +include ./aggregate/*.slt.part +include ./types/*.slt.part diff --git a/e2e_test/batch/functions/abs.slt b/e2e_test/batch/functions/abs.slt.part similarity index 100% rename from e2e_test/batch/functions/abs.slt rename to e2e_test/batch/functions/abs.slt.part diff --git a/e2e_test/batch/functions/concat.slt b/e2e_test/batch/functions/concat.slt.part similarity index 100% rename from e2e_test/batch/functions/concat.slt rename to e2e_test/batch/functions/concat.slt.part diff --git a/e2e_test/batch/functions/round.slt b/e2e_test/batch/functions/round.slt.part similarity index 100% rename from e2e_test/batch/functions/round.slt rename to e2e_test/batch/functions/round.slt.part diff --git a/e2e_test/batch/functions/to_char.slt b/e2e_test/batch/functions/to_char.slt.part similarity index 100% rename from e2e_test/batch/functions/to_char.slt rename to e2e_test/batch/functions/to_char.slt.part diff --git a/e2e_test/batch/local_mode.slt b/e2e_test/batch/local_mode.slt index ee067f7a891fe..7ece8f4d49201 100644 --- a/e2e_test/batch/local_mode.slt +++ b/e2e_test/batch/local_mode.slt @@ -5,6 +5,8 @@ statement ok SET QUERY_MODE TO local; include ./basic/*.slt.part +include ./aggregate/*.slt.part +include ./types/*.slt.part statement ok SET QUERY_MODE TO distributed; diff --git a/e2e_test/batch/types/arithmetic.slt b/e2e_test/batch/types/arithmetic.slt.part similarity index 100% rename from e2e_test/batch/types/arithmetic.slt rename to e2e_test/batch/types/arithmetic.slt.part diff --git a/e2e_test/batch/types/array_ty.slt b/e2e_test/batch/types/array_ty.slt.part similarity index 100% rename from e2e_test/batch/types/array_ty.slt rename to e2e_test/batch/types/array_ty.slt.part diff --git a/e2e_test/batch/types/boolean.slt b/e2e_test/batch/types/boolean.slt.part similarity index 100% rename from e2e_test/batch/types/boolean.slt rename to e2e_test/batch/types/boolean.slt.part diff --git a/e2e_test/batch/types/cast.slt b/e2e_test/batch/types/cast.slt.part similarity index 100% rename from e2e_test/batch/types/cast.slt rename to e2e_test/batch/types/cast.slt.part diff --git a/e2e_test/batch/types/decimal.slt b/e2e_test/batch/types/decimal.slt.part similarity index 100% rename from e2e_test/batch/types/decimal.slt rename to e2e_test/batch/types/decimal.slt.part diff --git a/e2e_test/batch/types/interval.slt b/e2e_test/batch/types/interval.slt.part similarity index 69% rename from e2e_test/batch/types/interval.slt rename to e2e_test/batch/types/interval.slt.part index e07549b934878..1d661d2a75e32 100644 --- a/e2e_test/batch/types/interval.slt +++ b/e2e_test/batch/types/interval.slt.part @@ -31,4 +31,24 @@ SELECT interval '5 minute', interval '2 m'; query TTTTT SELECT interval '6 second'; ---- -00:00:06 \ No newline at end of file +00:00:06 + +query T +SELECT interval '1' month = interval '30' day; +---- +t + +query T +SELECT interval '1' day = interval '24' hour; +---- +t + +query T +SELECT interval '1' day = interval '86400' second; +---- +t + +query T +SELECT interval '1' day - interval '12' hour = interval '12' hour; +---- +t \ No newline at end of file diff --git a/e2e_test/batch/types/struct_ty.slt b/e2e_test/batch/types/struct_ty.slt.part similarity index 100% rename from e2e_test/batch/types/struct_ty.slt rename to e2e_test/batch/types/struct_ty.slt.part diff --git a/e2e_test/batch/types/time.slt b/e2e_test/batch/types/time.slt deleted file mode 100644 index 7d064a0ffcbba..0000000000000 --- a/e2e_test/batch/types/time.slt +++ /dev/null @@ -1,4 +0,0 @@ -query T -values(extract(hour from timestamp '2001-02-16 20:38:40')); ----- -20 diff --git a/e2e_test/batch/types/time.slt.part b/e2e_test/batch/types/time.slt.part new file mode 100644 index 0000000000000..7ce58db19eb79 --- /dev/null +++ b/e2e_test/batch/types/time.slt.part @@ -0,0 +1,9 @@ +query T +values(extract(hour from timestamp '2001-02-16 20:38:40')); +---- +20 + +query TTTTT +select timestamp '2001-03-16 23:38:45' - timestamp '2001-02-16 20:38:40'; +---- +28 days 03:00:05 \ No newline at end of file diff --git a/proto/catalog.proto b/proto/catalog.proto index a9240e59e2b47..1bd8207ee35c5 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -2,6 +2,7 @@ syntax = "proto3"; package catalog; +import "common.proto"; import "plan_common.proto"; option optimize_for = SPEED; @@ -58,6 +59,7 @@ message Table { repeated int32 pk = 13; bool appendonly = 14; string owner = 15; + common.ParallelUnitMapping mapping = 16; } message Schema { diff --git a/proto/common.proto b/proto/common.proto index 1178a3919aa3c..757716805ab2d 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -61,6 +61,15 @@ message Cluster { map config = 3; } +message Buffer { + enum CompressionType { + INVALID = 0; + NONE = 1; + } + CompressionType compression = 1; + bytes body = 2; +} + // Vnode mapping for stream fragments / relational state tables. Stores mapping from virtual node to parallel unit id. message ParallelUnitMapping { uint32 table_id = 1; @@ -71,5 +80,5 @@ message ParallelUnitMapping { // Bitmap that records whether vnodes are present. message VNodeBitmap { uint32 table_id = 1; - bytes bitmap = 2; + Buffer bitmap = 2; } diff --git a/proto/data.proto b/proto/data.proto index cc998489a5b8d..897967175f45a 100644 --- a/proto/data.proto +++ b/proto/data.proto @@ -13,15 +13,6 @@ message IntervalUnit { int64 ms = 3; } -message Buffer { - enum CompressionType { - INVALID = 0; - NONE = 1; - } - CompressionType compression = 1; - bytes body = 2; -} - message DataType { enum IntervalType { INVALID = 0; @@ -102,8 +93,8 @@ enum ArrayType { message Array { ArrayType array_type = 1; - Buffer null_bitmap = 2; - repeated Buffer values = 3; + common.Buffer null_bitmap = 2; + repeated common.Buffer values = 3; StructArrayData struct_array_data = 4; ListArrayData list_array_data = 5; } diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 497630af9beb8..84ab148e03e29 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -108,6 +108,14 @@ message DropMaterializedSourceResponse { uint64 version = 2; } +// Used by risectl (and in the future, dashboard) +message ListMaterializedViewRequest {} + +// Used by risectl (and in the future, dashboard) +message ListMaterializedViewResponse { + repeated catalog.Table tables = 1; +} + service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); @@ -119,4 +127,5 @@ service DdlService { rpc DropMaterializedView(DropMaterializedViewRequest) returns (DropMaterializedViewResponse); rpc CreateMaterializedSource(CreateMaterializedSourceRequest) returns (CreateMaterializedSourceResponse); rpc DropMaterializedSource(DropMaterializedSourceRequest) returns (DropMaterializedSourceResponse); + rpc ListMaterializedView(ListMaterializedViewRequest) returns (ListMaterializedViewResponse); } diff --git a/proto/hummock.proto b/proto/hummock.proto index dda1b96da4924..b95ef71e60407 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -20,7 +20,7 @@ message SstableInfo { uint64 id = 1; KeyRange key_range = 2; uint64 file_size = 3; - repeated common.VNodeBitmap vnode_bitmaps = 4; + repeated uint32 table_ids = 4; uint64 unit_id = 5; } @@ -42,8 +42,12 @@ message UncommittedEpoch { } message HummockVersion { + message Levels { + repeated Level levels = 1; + } uint64 id = 1; - repeated Level levels = 2; + // Levels of each compaction group + map levels = 2; uint64 max_committed_epoch = 4; // Snapshots with epoch less than the safe epoch have been GCed. // Reads against such an epoch will fail. @@ -227,6 +231,9 @@ message GetCompactionGroupsResponse { message TriggerManualCompactionRequest { uint64 compaction_group_id = 1; + KeyRange key_range = 2; + uint32 table_id = 3; + uint32 level = 4; } message TriggerManualCompactionResponse { diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 15c3c80163f8e..1037d66e8ecfb 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -2,6 +2,7 @@ syntax = "proto3"; package stream_plan; +import "common.proto"; import "data.proto"; import "expr.proto"; import "plan_common.proto"; @@ -301,7 +302,7 @@ message StreamActor { bool same_worker_node_as_upstream = 7; // Vnodes that the executors in this actor own. If this actor is the only actor in its fragment, `vnode_bitmap` // will be empty. - bytes vnode_bitmap = 8; + common.Buffer vnode_bitmap = 8; } enum FragmentType { diff --git a/proto/stream_service.proto b/proto/stream_service.proto index f54f35159e868..c62511018c8dc 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -86,7 +86,11 @@ message BarrierCompleteResponse { string request_id = 1; common.Status status = 2; repeated CreateMviewProgress create_mview_progress = 3; - repeated hummock.SstableInfo sycned_sstables = 4; + message GroupedSstableInfo { + uint64 compaction_group_id = 1; + hummock.SstableInfo sst = 2; + } + repeated GroupedSstableInfo sycned_sstables = 4; } // Before starting streaming, the leader node broadcast the actor-host table to needed workers. diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index f3a9ebe6cc76f..392ca412e4c6e 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -21,7 +21,6 @@ use risingwave_common::array::{DataChunk, Row}; use risingwave_common::catalog::{ColumnDesc, OrderedColumnDesc, Schema, TableId}; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::Datum; -use risingwave_common::util::ordered::OrderedRowSerializer; use risingwave_common::util::sort_util::OrderType; use risingwave_expr::expr::LiteralExpression; use risingwave_pb::batch_plan::plan_node::NodeBody; @@ -150,7 +149,6 @@ impl BoxedExecutorBuilder for RowSeqScanExecutorBuilder { let pk_descs_proto = &seq_scan_node.table_desc.as_ref().unwrap().order_key; let pk_descs: Vec = pk_descs_proto.iter().map(|d| d.into()).collect(); let order_types: Vec = pk_descs.iter().map(|desc| desc.order).collect(); - let ordered_row_serializer = OrderedRowSerializer::new(order_types); let scan_range = seq_scan_node.scan_range.as_ref().unwrap(); let (pk_prefix_value, next_col_bounds) = get_scan_bound(scan_range.clone()); @@ -158,12 +156,7 @@ impl BoxedExecutorBuilder for RowSeqScanExecutorBuilder { dispatch_state_store!(source.context().try_get_state_store()?, state_store, { let keyspace = Keyspace::table_root(state_store.clone(), &table_id); let batch_stats = source.context().stats(); - let table = CellBasedTable::new( - keyspace.clone(), - column_descs, - Some(ordered_row_serializer), - None, - ); + let table = CellBasedTable::new(keyspace.clone(), column_descs, order_types, None); let scan_type = if pk_prefix_value.size() == 0 && is_full_range(&next_col_bounds) { let iter = table.batch_dedup_pk_iter(source.epoch, &pk_descs).await?; diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index b054481833976..229873a71b1ee 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -125,9 +125,10 @@ impl TaskService for BatchServiceImpl { let plan = req.get_plan().expect("no plan found").clone(); let epoch = req.epoch; let context = ComputeNodeContext::new(self.env.clone()); - debug!( + trace!( "local execute request: plan:{:?} with task id:{:?}", - plan, task_id + plan, + task_id ); let task = BatchTaskExecution::new(task_id, plan, context, epoch)?; let task = Arc::new(task); diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index b14b77f87e050..d3b268c0b9766 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -293,7 +293,7 @@ impl BatchTaskExecution { sender.send(Some(data_chunk?)).await?; } None => { - debug!("data chunk stream shuts down"); + trace!("data chunk stream shuts down"); sender.send(None).await?; break; } diff --git a/src/cmd/Cargo.toml b/src/cmd/Cargo.toml index 8e540c9c5e794..4550681824320 100644 --- a/src/cmd/Cargo.toml +++ b/src/cmd/Cargo.toml @@ -7,6 +7,7 @@ edition = "2021" static-link = ["workspace-static-link"] [dependencies] +anyhow = "1" clap = { version = "3", features = ["derive"] } log = { version = "0.4", features = ["release_max_level_info"] } madsim = "=0.2.0-alpha.3" diff --git a/src/cmd/src/bin/ctl.rs b/src/cmd/src/bin/ctl.rs index 1bdcea5c3f407..4105a82b8e84d 100644 --- a/src/cmd/src/bin/ctl.rs +++ b/src/cmd/src/bin/ctl.rs @@ -14,6 +14,7 @@ #![cfg_attr(coverage, feature(no_coverage))] +use anyhow::Result; use tikv_jemallocator::Jemalloc; #[global_allocator] @@ -21,7 +22,7 @@ static GLOBAL: Jemalloc = Jemalloc; #[cfg_attr(coverage, no_coverage)] #[tokio::main(flavor = "multi_thread", worker_threads = 4)] -async fn main() { +async fn main() -> Result<()> { use clap::StructOpt; let opts = risingwave_ctl::CliOpts::parse(); diff --git a/src/cmd_all/src/bin/risingwave.rs b/src/cmd_all/src/bin/risingwave.rs index 63209c0811169..f4656b71875ee 100644 --- a/src/cmd_all/src/bin/risingwave.rs +++ b/src/cmd_all/src/bin/risingwave.rs @@ -25,14 +25,16 @@ use std::env; use std::future::Future; use std::pin::Pin; +use anyhow::{bail, Result}; use clap::StructOpt; use risingwave_cmd_all::playground; -type RwFns = HashMap<&'static str, Box) -> Box>>>; +type RwFns = + HashMap<&'static str, Box) -> Box>>>>; #[cfg_attr(coverage, no_coverage)] #[tokio::main] -async fn main() { +async fn main() -> Result<()> { let mut fns: RwFns = HashMap::new(); // compute node configuration @@ -48,7 +50,9 @@ async fn main() { risingwave_logging::oneshot_common(); risingwave_logging::init_risingwave_logger(opts.enable_jaeger_tracing, false); - risingwave_compute::start(opts).await + risingwave_compute::start(opts).await; + + Ok(()) }) }), ); @@ -67,7 +71,9 @@ async fn main() { risingwave_logging::oneshot_common(); risingwave_logging::init_risingwave_logger(false, false); - risingwave_meta::start(opts).await + risingwave_meta::start(opts).await; + + Ok(()) }) }), ); @@ -86,7 +92,9 @@ async fn main() { risingwave_logging::oneshot_common(); risingwave_logging::init_risingwave_logger(false, false); - risingwave_frontend::start(opts).await + risingwave_frontend::start(opts).await; + + Ok(()) }) }), ); @@ -105,7 +113,9 @@ async fn main() { risingwave_logging::oneshot_common(); risingwave_logging::init_risingwave_logger(false, false); - risingwave_compactor::start(opts).await + risingwave_compactor::start(opts).await; + + Ok(()) }) }), ); @@ -131,7 +141,7 @@ async fn main() { for fn_name in ["play", "playground"] { fns.insert( fn_name, - Box::new(|_: Vec| Box::new(async move { playground().await.unwrap() })), + Box::new(|_: Vec| Box::new(async move { playground().await })), ); } @@ -158,11 +168,13 @@ async fn main() { match fns.remove(target.as_str()) { Some(func) => { - let func: Pin>> = func(args).into(); - func.await + let func: Pin>>> = func(args).into(); + func.await?; } None => { - panic!("unknown target: {}\nPlease either:\n* set `RW_NODE` env variable (`RW_NODE=`)\n* create a symbol link to `risingwave` binary (ln -s risingwave )\n* start with subcommand `risingwave ``\nwith one of the following: {:?}", target, fns.keys().collect::>()); + bail!("unknown target: {}\nPlease either:\n* set `RW_NODE` env variable (`RW_NODE=`)\n* create a symbol link to `risingwave` binary (ln -s risingwave )\n* start with subcommand `risingwave ``\nwith one of the following: {:?}", target, fns.keys().collect::>()); } } + + Ok(()) } diff --git a/src/common/src/array/decimal_array.rs b/src/common/src/array/decimal_array.rs index cac15736df2b4..f11fad4e02c6f 100644 --- a/src/common/src/array/decimal_array.rs +++ b/src/common/src/array/decimal_array.rs @@ -16,8 +16,9 @@ use std::hash::{Hash, Hasher}; use std::mem::size_of; use itertools::Itertools; -use risingwave_pb::data::buffer::CompressionType; -use risingwave_pb::data::{Array as ProstArray, ArrayType, Buffer}; +use risingwave_pb::common::buffer::CompressionType; +use risingwave_pb::common::Buffer; +use risingwave_pb::data::{Array as ProstArray, ArrayType}; use super::{Array, ArrayBuilder, ArrayIterator, ArrayResult, NULL_VAL_FOR_HASH}; use crate::array::{ArrayBuilderImpl, ArrayMeta}; diff --git a/src/common/src/array/primitive_array.rs b/src/common/src/array/primitive_array.rs index 12c751df80096..982e8051cd67f 100644 --- a/src/common/src/array/primitive_array.rs +++ b/src/common/src/array/primitive_array.rs @@ -17,8 +17,9 @@ use std::hash::{Hash, Hasher}; use std::io::Write; use std::mem::size_of; -use risingwave_pb::data::buffer::CompressionType; -use risingwave_pb::data::{Array as ProstArray, ArrayType, Buffer}; +use risingwave_pb::common::buffer::CompressionType; +use risingwave_pb::common::Buffer; +use risingwave_pb::data::{Array as ProstArray, ArrayType}; use super::{Array, ArrayBuilder, ArrayIterator, ArrayResult, NULL_VAL_FOR_HASH}; use crate::array::{ArrayBuilderImpl, ArrayImpl, ArrayMeta}; diff --git a/src/common/src/array/utf8_array.rs b/src/common/src/array/utf8_array.rs index b39f050fc3319..c510cec0223bd 100644 --- a/src/common/src/array/utf8_array.rs +++ b/src/common/src/array/utf8_array.rs @@ -17,8 +17,9 @@ use std::iter; use std::mem::size_of; use itertools::Itertools; -use risingwave_pb::data::buffer::CompressionType; -use risingwave_pb::data::{Array as ProstArray, ArrayType, Buffer}; +use risingwave_pb::common::buffer::CompressionType; +use risingwave_pb::common::Buffer; +use risingwave_pb::data::{Array as ProstArray, ArrayType}; use super::{Array, ArrayBuilder, ArrayIterator, ArrayMeta, ArrayResult, NULL_VAL_FOR_HASH}; use crate::array::ArrayBuilderImpl; diff --git a/src/common/src/buffer/bitmap.rs b/src/common/src/buffer/bitmap.rs index 72a8995d83202..29d4119f12f2d 100644 --- a/src/common/src/buffer/bitmap.rs +++ b/src/common/src/buffer/bitmap.rs @@ -37,8 +37,8 @@ use std::ops::{BitAnd, BitOr}; use bytes::Bytes; use itertools::Itertools; -use risingwave_pb::data::buffer::CompressionType; -use risingwave_pb::data::Buffer as ProstBuffer; +use risingwave_pb::common::buffer::CompressionType; +use risingwave_pb::common::Buffer as ProstBuffer; use crate::array::error::ArrayError; use crate::array::{Array, ArrayResult, BoolArray}; @@ -64,6 +64,33 @@ impl BitmapBuilder { } } + pub fn zeroed(len: usize) -> BitmapBuilder { + BitmapBuilder { + len, + data: vec![0; len / 8], + num_high_bits: 0, + head: 0, + } + } + + pub fn set(&mut self, n: usize, val: bool) { + assert!(n < self.len); + + let byte = self.data.get_mut(n / 8).unwrap_or(&mut self.head); + let mask = 1 << (n % 8); + match (*byte & mask > 0, val) { + (true, false) => { + *byte &= !mask; + self.num_high_bits -= 1; + } + (false, true) => { + *byte |= mask; + self.num_high_bits += 1; + } + _ => {} + } + } + pub fn append(&mut self, bit_set: bool) -> &mut Self { self.head |= (bit_set as u8) << (self.len % 8); self.num_high_bits += bit_set as usize; @@ -537,4 +564,27 @@ mod tests { let bm2 = (vec![false]).try_into().unwrap(); assert_eq!(bm1, bm2); } + + #[test] + fn test_bitmap_set() { + let mut b = BitmapBuilder::zeroed(10); + assert_eq!(b.num_high_bits, 0); + + b.set(0, true); + b.set(7, true); + b.set(8, true); + b.set(9, true); + assert_eq!(b.num_high_bits, 4); + + b.set(7, false); + b.set(8, false); + assert_eq!(b.num_high_bits, 2); + + b.append(true); + assert_eq!(b.len, 11); + assert_eq!(b.num_high_bits, 3); + + let b = b.finish(); + assert_eq!(b.bits.to_vec(), &[0b0000_0001, 0b0000_0110]); + } } diff --git a/src/common/src/hash/key.rs b/src/common/src/hash/key.rs index 57ccd78bc1acc..5937f2334428e 100644 --- a/src/common/src/hash/key.rs +++ b/src/common/src/hash/key.rs @@ -27,7 +27,8 @@ use crate::array::{ use crate::error::Result; use crate::types::{ DataType, Datum, Decimal, IntervalUnit, NaiveDateTimeWrapper, NaiveDateWrapper, - NaiveTimeWrapper, OrderedF32, OrderedF64, ScalarRef, ToOwnedDatum, + NaiveTimeWrapper, OrderedF32, OrderedF64, ScalarRef, ToOwnedDatum, VirtualNode, + VIRTUAL_NODE_COUNT, }; use crate::util::hash_util::CRC32FastBuilder; @@ -40,11 +41,6 @@ use crate::util::hash_util::CRC32FastBuilder; /// are encoded from both `t.b, t.c`. If t.b="abc", t.c=1, the hashkey may be /// encoded in certain format of ("abc", 1). -pub type VirtualNode = u16; -pub const VNODE_BITS: usize = 11; -pub const VIRTUAL_NODE_COUNT: usize = 1 << VNODE_BITS; -pub const VNODE_BITMAP_LEN: usize = 1 << (VNODE_BITS - 3); - /// A wrapper for u64 hash result. #[derive(Default, Clone, Debug, PartialEq)] pub struct HashCode(pub u64); diff --git a/src/common/src/types/interval.rs b/src/common/src/types/interval.rs index f69775c75d2eb..4606fde08b321 100644 --- a/src/common/src/types/interval.rs +++ b/src/common/src/types/interval.rs @@ -12,7 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cmp::Ordering; use std::fmt::{Display, Formatter}; +use std::hash::{Hash, Hasher}; use std::io::Write; use std::ops::{Add, Sub}; @@ -34,17 +36,16 @@ use super::*; /// One month may contain 28/31 days. One day may contain 23/25 hours. /// This internals is learned from PG: /// -/// -/// FIXME: if this derives `PartialEq` and `PartialOrd`, caller must guarantee the fields are valid. -#[derive( - Debug, Clone, Copy, Eq, PartialEq, PartialOrd, Ord, Hash, Default, Serialize, Deserialize, -)] +/// FIXME: the comparison of memcomparable encoding will be just compare these three numbers. +#[derive(Debug, Clone, Copy, Default, Serialize, Deserialize)] pub struct IntervalUnit { months: i32, days: i32, ms: i64, } +const DAY_MS: i64 = 86400000; + impl IntervalUnit { pub fn new(months: i32, days: i32, ms: i64) -> Self { IntervalUnit { months, days, ms } @@ -95,6 +96,17 @@ impl IntervalUnit { } } + /// Justify interval, convert 1 month to 30 days and 86400 ms to 1 day. + /// If day is positive, complement the ms negative value. + /// These rules only use in interval comparison. + pub fn justify_interval(&mut self) { + let month = (self.months * 30) as i64 * DAY_MS; + self.ms = self.ms + month + (self.days) as i64 * DAY_MS; + self.days = (self.ms / DAY_MS) as i32; + self.ms %= DAY_MS; + self.months = 0; + } + #[must_use] pub fn negative(&self) -> Self { IntervalUnit { @@ -236,6 +248,46 @@ impl Add for IntervalUnit { } } +impl PartialOrd for IntervalUnit { + fn partial_cmp(&self, other: &Self) -> Option { + if self.eq(other) { + Some(Ordering::Equal) + } else { + let diff = *self - *other; + let days = (diff.months * 30 + diff.days) as i64; + Some((days * DAY_MS + diff.ms).cmp(&0)) + } + } +} + +impl Hash for IntervalUnit { + fn hash(&self, state: &mut H) { + let mut interval = *self; + interval.justify_interval(); + interval.months.hash(state); + interval.ms.hash(state); + interval.days.hash(state); + } +} + +impl PartialEq for IntervalUnit { + fn eq(&self, other: &Self) -> bool { + let mut interval = *self; + interval.justify_interval(); + let mut other = *other; + other.justify_interval(); + interval.days == other.days && interval.ms == other.ms + } +} + +impl Eq for IntervalUnit {} + +impl Ord for IntervalUnit { + fn cmp(&self, other: &Self) -> Ordering { + self.partial_cmp(other).unwrap() + } +} + impl CheckedAdd for IntervalUnit { fn checked_add(&self, other: &Self) -> Option { let months = self.months.checked_add(other.months)?; diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index e401d24d09eeb..73a065d408b63 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -57,6 +57,15 @@ use crate::array::{ StructValue, }; +/// Parallel unit is the minimal scheduling unit. +pub type ParallelUnitId = u32; + +// VirtualNode (a.k.a. VNode) is a minimal partition that a set of keys belong to. It is used for +// consistent hashing. +pub type VirtualNode = u16; +pub const VNODE_BITS: usize = 11; +pub const VIRTUAL_NODE_COUNT: usize = 1 << VNODE_BITS; + pub type OrderedF32 = ordered_float::OrderedFloat; pub type OrderedF64 = ordered_float::OrderedFloat; diff --git a/src/common/src/util/ordered/serde.rs b/src/common/src/util/ordered/serde.rs index f62a0cbbb22e3..bb5db618dc989 100644 --- a/src/common/src/util/ordered/serde.rs +++ b/src/common/src/util/ordered/serde.rs @@ -75,6 +75,7 @@ impl OrderedRowSerializer { Self { order_types } } + #[must_use] pub fn prefix(&self, len: usize) -> Self { Self { order_types: self.order_types[..len].to_vec(), @@ -114,10 +115,6 @@ impl OrderedRowSerializer { append_to.extend(serializer.into_inner()); } } - - pub fn into_order_types(self) -> Vec { - self.order_types - } } /// Deserializer of the `Row`. diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index d72a6ec6de039..94aa0161dce32 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -21,6 +21,7 @@ itertools = "0.10" lazy_static = "1" log = "0.4" madsim = "=0.2.0-alpha.3" +maplit = "1.0.2" memcomparable = { path = "../utils/memcomparable" } num-traits = "0.2" paste = "1" @@ -29,6 +30,7 @@ prost = "0.10" rdkafka = { version = "0.28", features = ["cmake-build"] } risingwave_batch = { path = "../batch" } risingwave_common = { path = "../common" } +risingwave_connector = { path = "../connector" } risingwave_pb = { path = "../prost" } risingwave_rpc_client = { path = "../rpc_client" } risingwave_source = { path = "../source" } diff --git a/src/compute/src/rpc/service/stream_service.rs b/src/compute/src/rpc/service/stream_service.rs index 621521c171813..b9758b065c718 100644 --- a/src/compute/src/rpc/service/stream_service.rs +++ b/src/compute/src/rpc/service/stream_service.rs @@ -18,6 +18,7 @@ use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::error::{tonic_err, Result as RwResult}; use risingwave_pb::catalog::Source; +use risingwave_pb::stream_service::barrier_complete_response::GroupedSstableInfo; use risingwave_pb::stream_service::stream_service_server::StreamService; use risingwave_pb::stream_service::*; use risingwave_stream::executor::{Barrier, Epoch}; @@ -160,7 +161,14 @@ impl StreamService for StreamServiceImpl { request_id: req.request_id, status: None, create_mview_progress: collect_result.create_mview_progress, - sycned_sstables: collect_result.synced_sstables, + sycned_sstables: collect_result + .synced_sstables + .into_iter() + .map(|(compaction_group_id, sst)| GroupedSstableInfo { + compaction_group_id, + sst: Some(sst), + }) + .collect_vec(), })) } diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index a9bed1aede845..202c9df397aa5 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -201,7 +201,12 @@ async fn test_table_v2_materialize() -> Result<()> { // Since we have not polled `Materialize`, we cannot scan anything from this table let keyspace = Keyspace::table_root(memory_state_store, &source_table_id); - let table = CellBasedTable::new(keyspace, column_descs.clone(), None, None); + let table = CellBasedTable::new( + keyspace, + column_descs.clone(), + vec![OrderType::Ascending], + None, + ); let ordered_column_descs: Vec = column_descs .iter() @@ -395,7 +400,12 @@ async fn test_row_seq_scan() -> Result<()> { None, vec![0_usize], ); - let table = CellBasedTable::new(keyspace, column_descs.clone(), None, None); + let table = CellBasedTable::new( + keyspace, + column_descs.clone(), + vec![OrderType::Ascending], + None, + ); let epoch: u64 = 0; diff --git a/src/connector/src/datagen/source/reader.rs b/src/connector/src/datagen/source/reader.rs index 58003bea34fa6..88c5515db71ff 100644 --- a/src/connector/src/datagen/source/reader.rs +++ b/src/connector/src/datagen/source/reader.rs @@ -104,10 +104,20 @@ impl SplitReader for DatagenSplitReader { .get(&random_seed_key) .map(|s| s.to_string()) { - Some(seed) => seed.parse::().unwrap_or(split_index), + Some(seed) => { + match seed.parse::() { + // we use given seed xor split_index to make sure every split has different + // seed + Ok(seed) => seed ^ split_index, + Err(e) => { + log::warn!("cannot parse {:?} to u64 due to {:?}, will use {:?} as random seed", seed, e, split_index); + split_index + } + } + } None => split_index, }; - match column.data_type{ + match column.data_type { DataType::Timestamp => { let max_past_key = format!("fields.{}.max_past", name); let max_past_value = diff --git a/src/connector/src/datagen/split.rs b/src/connector/src/datagen/split.rs index 16e2e6412f741..aea0e3939132c 100644 --- a/src/connector/src/datagen/split.rs +++ b/src/connector/src/datagen/split.rs @@ -20,9 +20,9 @@ use crate::base::SplitMetaData; #[derive(Clone, Serialize, Deserialize, Debug, Default, PartialEq, Hash)] pub struct DatagenSplit { - pub(crate) split_index: i32, - pub(crate) split_num: i32, - pub(crate) start_offset: Option, + pub split_index: i32, + pub split_num: i32, + pub start_offset: Option, } impl SplitMetaData for DatagenSplit { diff --git a/src/connector/src/lib.rs b/src/connector/src/lib.rs index 8a64073cabc9a..c28ff81ae4e6e 100644 --- a/src/connector/src/lib.rs +++ b/src/connector/src/lib.rs @@ -37,9 +37,9 @@ extern crate core; pub mod base; -mod datagen; +pub mod datagen; mod filesystem; -mod kafka; +pub mod kafka; pub mod kinesis; mod nexmark; mod pulsar; diff --git a/src/ctl/Cargo.toml b/src/ctl/Cargo.toml index e7254131c40b5..62fa3dc9cb650 100644 --- a/src/ctl/Cargo.toml +++ b/src/ctl/Cargo.toml @@ -7,10 +7,21 @@ edition = "2021" anyhow = "1" bytes = "1" clap = { version = "3", features = ["derive"] } +futures = { version = "0.3", default-features = false, features = ["alloc"] } risingwave_common = { path = "../common" } +risingwave_frontend = { path = "../frontend" } risingwave_hummock_sdk = { path = "../storage/hummock_sdk" } risingwave_pb = { path = "../prost" } risingwave_rpc_client = { path = "../rpc_client" } risingwave_storage = { path = "../storage" } +tokio = { version = "=0.2.0-alpha.3", package = "madsim-tokio", features = [ + "rt", + "rt-multi-thread", + "sync", + "macros", + "time", + "signal", + "fs", +] } tracing = { version = "0.1" } workspace-hack = { version = "0.1", path = "../workspace-hack" } diff --git a/src/ctl/src/cmd_impl.rs b/src/ctl/src/cmd_impl.rs index f92af662de0ac..d6617c21b9f37 100644 --- a/src/ctl/src/cmd_impl.rs +++ b/src/ctl/src/cmd_impl.rs @@ -12,4 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod bench; pub mod hummock; +pub mod table; diff --git a/src/ctl/src/cmd_impl/bench.rs b/src/ctl/src/cmd_impl/bench.rs new file mode 100644 index 0000000000000..309f71e61adff --- /dev/null +++ b/src/ctl/src/cmd_impl/bench.rs @@ -0,0 +1,13 @@ +// Copyright 2022 Singularity Data +// +// 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. diff --git a/src/ctl/src/cmd_impl/hummock.rs b/src/ctl/src/cmd_impl/hummock.rs index 49fd19ac8d9f0..9300686a54c10 100644 --- a/src/ctl/src/cmd_impl/hummock.rs +++ b/src/ctl/src/cmd_impl/hummock.rs @@ -16,5 +16,7 @@ mod list_version; pub use list_version::*; mod list_kv; pub use list_kv::*; +mod sst_dump; +pub use sst_dump::*; mod trigger_manual_compaction; pub use trigger_manual_compaction::*; diff --git a/src/ctl/src/cmd_impl/hummock/list_kv.rs b/src/ctl/src/cmd_impl/hummock/list_kv.rs index 8d3d01bb3507c..377a143485202 100644 --- a/src/ctl/src/cmd_impl/hummock/list_kv.rs +++ b/src/ctl/src/cmd_impl/hummock/list_kv.rs @@ -18,18 +18,18 @@ use risingwave_storage::StateStore; use crate::common::HummockServiceOpts; -pub async fn list_kv(epoch: u64, table_id: u32) -> anyhow::Result<()> { +pub async fn list_kv(epoch: u64, table_id: Option) -> anyhow::Result<()> { let hummock_opts = HummockServiceOpts::from_env()?; - let hummock = hummock_opts.create_hummock_store().await?; + let (_, hummock) = hummock_opts.create_hummock_store().await?; if epoch == u64::MAX { tracing::info!("using u64::MAX as epoch"); } let scan_result = match table_id { - u32::MAX => { + None => { tracing::info!("using .. as range"); hummock.scan::<_, Vec>(.., None, u64::MAX).await? } - _ => { + Some(table_id) => { let mut buf = BytesMut::with_capacity(5); buf.put_u8(b't'); buf.put_u32(table_id); @@ -41,19 +41,19 @@ pub async fn list_kv(epoch: u64, table_id: u32) -> anyhow::Result<()> { let print_string = match k[0] { b't' => { let mut buf = &k[1..]; - format!("table_id:{:?}", buf.get_u32()) + format!("[t{}]", buf.get_u32()) // table id } b's' => { let mut buf = &k[1..]; - format!("shared_executor_id:{:?}", buf.get_u64()) + format!("[s{}]", buf.get_u64()) // shared executor root } b'e' => { let mut buf = &k[1..]; - format!("executor_id:{:?}", buf.get_u64()) + format!("[e{}]", buf.get_u64()) // executor id } _ => "no title".to_string(), }; - println!("{}\n key : {:?} ====> value : {:?}", print_string, k, v) + println!("{} {:?} => {:?}", print_string, k, v) } Ok(()) diff --git a/src/ctl/src/cmd_impl/hummock/sst_dump.rs b/src/ctl/src/cmd_impl/hummock/sst_dump.rs new file mode 100644 index 0000000000000..5a8271f500e3e --- /dev/null +++ b/src/ctl/src/cmd_impl/hummock/sst_dump.rs @@ -0,0 +1,76 @@ +// Copyright 2022 Singularity Data +// +// 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_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; +use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; +use risingwave_rpc_client::HummockMetaClient; +use risingwave_storage::monitor::StoreLocalStatistic; + +use crate::common::HummockServiceOpts; + +pub async fn sst_dump() -> anyhow::Result<()> { + // Retrieves the SSTable store so we can access the SSTableMeta + let hummock_opts = HummockServiceOpts::from_env()?; + let (meta_client, hummock) = hummock_opts.create_hummock_store().await?; + let sstable_store = &*hummock.inner().sstable_store(); + + // Retrieves the latest HummockVersion from the meta client so we can access the SSTableInfo + let version = meta_client.pin_version(u64::MAX).await?; + + // TODO #2065: iterate all compaction groups + for level in version + .get_compaction_group_levels(StaticCompactionGroupId::StateDefault.into()) + .clone() + { + for sstable_info in level.table_infos { + let id = sstable_info.id; + + let sstable_cache = sstable_store + .sstable(id, &mut StoreLocalStatistic::default()) + .await?; + let sstable_meta = &sstable_cache.value().meta; + + println!("SST id: {}", id); + println!("-------------------------------------"); + println!("File Size: {}", sstable_info.file_size); + + if let Some(key_range) = sstable_info.key_range { + println!("Key Range:"); + println!( + "\tleft:\t{:?}\n\tright:\t{:?}\n\tinf:\t{:?}", + key_range.left, key_range.right, key_range.inf + ); + } else { + println!("Key Range: None"); + } + + println!("Block Metadata:"); + for (i, block_meta) in sstable_meta.block_metas.iter().enumerate() { + println!( + "\tBlock {}, Size: {}, Offset: {}", + i, block_meta.len, block_meta.offset + ); + } + + println!("Estimated Table Size: {}", sstable_meta.estimated_size); + println!("Bloom Filter Size: {}", sstable_meta.bloom_filter.len()); + println!("Key Count: {}", sstable_meta.key_count); + println!("Version: {}", sstable_meta.version); + } + } + + meta_client.unpin_version(&[version.id]).await?; + + Ok(()) +} diff --git a/src/ctl/src/cmd_impl/hummock/trigger_manual_compaction.rs b/src/ctl/src/cmd_impl/hummock/trigger_manual_compaction.rs index 12d3afbca191f..07c82998e0af3 100644 --- a/src/ctl/src/cmd_impl/hummock/trigger_manual_compaction.rs +++ b/src/ctl/src/cmd_impl/hummock/trigger_manual_compaction.rs @@ -16,11 +16,15 @@ use risingwave_rpc_client::HummockMetaClient; use crate::common::MetaServiceOpts; -pub async fn trigger_manual_compaction(compaction_group_id: u64) -> anyhow::Result<()> { +pub async fn trigger_manual_compaction( + compaction_group_id: u64, + table_id: u32, + level: u32, +) -> anyhow::Result<()> { let meta_opts = MetaServiceOpts::from_env()?; let meta_client = meta_opts.create_meta_client().await?; let result = meta_client - .trigger_manual_compaction(compaction_group_id) + .trigger_manual_compaction(compaction_group_id, table_id, level) .await; println!("{:#?}", result); Ok(()) diff --git a/src/ctl/src/cmd_impl/table.rs b/src/ctl/src/cmd_impl/table.rs new file mode 100644 index 0000000000000..21d20cac1f4be --- /dev/null +++ b/src/ctl/src/cmd_impl/table.rs @@ -0,0 +1,16 @@ +// Copyright 2022 Singularity Data +// +// 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. + +mod scan; +pub use scan::*; diff --git a/src/ctl/src/cmd_impl/table/scan.rs b/src/ctl/src/cmd_impl/table/scan.rs new file mode 100644 index 0000000000000..c111b5f4ca12f --- /dev/null +++ b/src/ctl/src/cmd_impl/table/scan.rs @@ -0,0 +1,54 @@ +// Copyright 2022 Singularity Data +// +// 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 anyhow::{anyhow, Result}; +use futures::{pin_mut, StreamExt}; +use risingwave_common::catalog::TableId; +use risingwave_frontend::catalog::TableCatalog; +use risingwave_storage::table::state_table::StateTable; +use risingwave_storage::Keyspace; + +use crate::common::HummockServiceOpts; + +pub async fn scan(table_id: String) -> Result<()> { + let hummock_opts = HummockServiceOpts::from_env()?; + let (meta, hummock) = hummock_opts.create_hummock_store().await?; + let mvs = meta.list_materialize_view().await?; + let mv = mvs + .iter() + .find(|x| x.name == table_id) + .ok_or_else(|| anyhow!("mv not found"))? + .clone(); + println!("{:#?}", mv); + let table = TableCatalog::from(&mv); + let state_table = StateTable::new( + Keyspace::table_root(hummock, &TableId::new(mv.id)), + table + .columns() + .iter() + .map(|x| x.column_desc.clone()) + .collect(), + table.order_desc().iter().map(|x| x.order).collect(), + Some(table.pks.clone()), + table.distribution_keys().to_vec(), + ); + + let stream = state_table.iter(u64::MAX).await?; + pin_mut!(stream); + while let Some(item) = stream.next().await { + let item = item?; + println!("{:?}", item); + } + Ok(()) +} diff --git a/src/ctl/src/common/hummock_service.rs b/src/ctl/src/common/hummock_service.rs index 5fdf77c403526..de324be5f5e10 100644 --- a/src/ctl/src/common/hummock_service.rs +++ b/src/ctl/src/common/hummock_service.rs @@ -15,8 +15,9 @@ use std::env; use std::sync::Arc; -use anyhow::{anyhow, Result}; +use anyhow::{anyhow, bail, Result}; use risingwave_common::config::StorageConfig; +use risingwave_rpc_client::MetaClient; use risingwave_storage::hummock::hummock_meta_client::MonitoredHummockMetaClient; use risingwave_storage::hummock::HummockStorage; use risingwave_storage::monitor::{ @@ -39,22 +40,25 @@ impl HummockServiceOpts { /// * `RW_HUMMOCK_URL`: meta service address pub fn from_env() -> Result { let meta_opts = MetaServiceOpts::from_env()?; - let hummock_url = env::var("RW_HUMMOCK_URL").unwrap_or_else(|_| { - const DEFAULT_ADDR: &str = - "hummock+minio://hummockadmin:hummockadmin@127.0.0.1:9301/hummock001"; - tracing::warn!( - "`RW_HUMMOCK_URL` not found, using default hummock URL {}", - DEFAULT_ADDR - ); - DEFAULT_ADDR.to_string() - }); + + let hummock_url = match env::var("RW_HUMMOCK_URL") { + Ok(url) => { + tracing::info!("using Hummock URL from `RW_HUMMOCK_URL`: {}", url); + url + } + Err(_) => { + bail!("env variable `RW_HUMMOCK_URL` not found, please do one of the following:\n* use `./risedev ctl` to start risectl.\n* `source .risingwave/config/risectl-env` or `source ~/risingwave-deploy/risectl-env` before running risectl.\n* manually set `RW_HUMMOCK_URL` in env variable.\nPlease also remember to add `use: minio` to risedev config."); + } + }; Ok(Self { meta_opts, hummock_url, }) } - pub async fn create_hummock_store(&self) -> Result> { + pub async fn create_hummock_store( + &self, + ) -> Result<(MetaClient, MonitoredStateStore)> { let meta_client = self.meta_opts.create_meta_client().await?; // FIXME: allow specify custom config @@ -66,7 +70,7 @@ impl HummockServiceOpts { &self.hummock_url, Arc::new(config), Arc::new(MonitoredHummockMetaClient::new( - meta_client, + meta_client.clone(), Arc::new(HummockMetrics::unused()), )), Arc::new(StateStoreMetrics::unused()), @@ -75,7 +79,7 @@ impl HummockServiceOpts { .await?; if let StateStoreImpl::HummockStateStore(hummock_state_store) = state_store_impl { - Ok(hummock_state_store) + Ok((meta_client, hummock_state_store)) } else { Err(anyhow!("only Hummock state store is supported in risectl")) } diff --git a/src/ctl/src/common/meta_service.rs b/src/ctl/src/common/meta_service.rs index e899e9cda47c7..c7d75fd9f685e 100644 --- a/src/ctl/src/common/meta_service.rs +++ b/src/ctl/src/common/meta_service.rs @@ -14,7 +14,7 @@ use std::env; -use anyhow::Result; +use anyhow::{bail, Result}; use risingwave_pb::common::WorkerType; use risingwave_rpc_client::MetaClient; @@ -29,14 +29,15 @@ impl MetaServiceOpts { /// /// * `RW_META_ADDR`: meta service address pub fn from_env() -> Result { - let meta_addr = env::var("RW_META_ADDR").unwrap_or_else(|_| { - const DEFAULT_ADDR: &str = "http://127.0.0.1:5690"; - tracing::warn!( - "`RW_META_ADDR` not found, using default meta address {}", - DEFAULT_ADDR - ); - DEFAULT_ADDR.to_string() - }); + let meta_addr = match env::var("RW_META_ADDR") { + Ok(url) => { + tracing::info!("using meta addr from `RW_META_ADDR`: {}", url); + url + } + Err(_) => { + bail!("env variable `RW_META_ADDR` not found, please do one of the following:\n* use `./risedev ctl` to start risectl.\n* `source .risingwave/config/risectl-env` or `source ~/risingwave-deploy/risectl-env` before running risectl.\n* manually set `RW_META_ADDR` in env variable.\nrisectl requires a full persistent cluster to operate, so please also remember to add `use: minio` to risedev config."); + } + }; Ok(Self { meta_addr }) } diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index 5c10cf35a8d02..19dade21b4e8f 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -12,10 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::Result; use clap::{Parser, Subcommand}; mod cmd_impl; pub(crate) mod common; +/// risectl provides internal access to the RisingWave cluster. Generally, you will need +/// to provide the meta address and the state store URL to enable risectl to access the cluster. You +/// must start RisingWave in full cluster mode (e.g. enable MinIO and compactor in risedev.yml) +/// instead of playground mode to use this tool. risectl will read environment variables +/// `RW_META_ADDR` and `RW_HUMMOCK_URL` to configure itself. #[derive(Parser)] #[clap(author, version, about, long_about = None)] #[clap(propagate_version = true)] @@ -31,6 +37,9 @@ enum Commands { /// Commands for Hummock #[clap(subcommand)] Hummock(HummockCommands), + /// Commands for Benchmarks + #[clap(subcommand)] + Table(TableCommands), } #[derive(Subcommand)] @@ -42,27 +51,57 @@ enum HummockCommands { #[clap(short, long = "epoch", default_value_t = u64::MAX)] epoch: u64, - #[clap(short, long = "table-id", default_value_t = u32::MAX)] - tableid: u32, + #[clap(short, long = "table-id")] + table_id: Option, }, + SstDump, + /// trigger a targeted compaction through compaction_group_id TriggerManualCompaction { #[clap(short, long = "compaction-group-id", default_value_t = 2)] compaction_group_id: u64, + + #[clap(short, long = "table-id", default_value_t = 0)] + table_id: u32, + + #[clap(short, long = "level", default_value_t = 1)] + level: u32, }, } -pub async fn start(opts: CliOpts) { - match &opts.command { +#[derive(Subcommand)] +enum TableCommands { + /// benchmark state table + Scan { + /// name of the materialized view to operate on + #[clap()] + mv_name: String, + }, +} + +pub async fn start(opts: CliOpts) -> Result<()> { + match opts.command { Commands::Hummock(HummockCommands::ListVersion) => { - cmd_impl::hummock::list_version().await.unwrap() + tokio::spawn(cmd_impl::hummock::list_version()).await??; } - Commands::Hummock(HummockCommands::ListKv { epoch, tableid }) => { - cmd_impl::hummock::list_kv(*epoch, *tableid).await.unwrap() + Commands::Hummock(HummockCommands::ListKv { epoch, table_id }) => { + tokio::spawn(cmd_impl::hummock::list_kv(epoch, table_id)).await??; } + Commands::Hummock(HummockCommands::SstDump) => cmd_impl::hummock::sst_dump().await.unwrap(), Commands::Hummock(HummockCommands::TriggerManualCompaction { compaction_group_id, - }) => cmd_impl::hummock::trigger_manual_compaction(*compaction_group_id) - .await - .unwrap(), + table_id, + level, + }) => { + tokio::spawn(cmd_impl::hummock::trigger_manual_compaction( + compaction_group_id, + table_id, + level, + )) + .await?? + } + Commands::Table(TableCommands::Scan { mv_name }) => { + tokio::spawn(cmd_impl::table::scan(mv_name)).await?? + } } + Ok(()) } diff --git a/src/expr/src/expr/template.rs b/src/expr/src/expr/template.rs index f6ef2eeef7227..7d7062f5f031d 100644 --- a/src/expr/src/expr/template.rs +++ b/src/expr/src/expr/template.rs @@ -382,6 +382,7 @@ macro_rules! for_all_cmp_variants { { float32, decimal, float64, $general_f }, { float64, decimal, float64, $general_f }, { timestamp, timestamp, timestamp, $general_f }, + { interval, interval, interval, $general_f }, { date, date, date, $general_f }, { boolean, boolean, boolean, $general_f }, { timestamp, date, timestamp, $general_f }, diff --git a/src/expr/src/vector_op/arithmetic_op.rs b/src/expr/src/vector_op/arithmetic_op.rs index 51918962b9dfb..c8973ef42a19f 100644 --- a/src/expr/src/vector_op/arithmetic_op.rs +++ b/src/expr/src/vector_op/arithmetic_op.rs @@ -15,7 +15,9 @@ use std::any::type_name; use std::convert::TryInto; use std::fmt::Debug; +use std::ops::Sub; +use chrono::Duration; use num_traits::{CheckedDiv, CheckedMul, CheckedNeg, CheckedRem, CheckedSub, Signed}; use risingwave_common::types::{ CheckedAdd, Decimal, IntervalUnit, NaiveDateTimeWrapper, NaiveDateWrapper, @@ -125,7 +127,9 @@ pub fn timestamp_timestamp_sub( r: NaiveDateTimeWrapper, ) -> Result { let tmp = l.0 - r.0; - Ok(IntervalUnit::new(0, tmp.num_days() as i32, 0)) + let days = tmp.num_days(); + let ms = tmp.sub(Duration::days(tmp.num_days())).num_milliseconds(); + Ok(IntervalUnit::new(0, days as i32, ms)) } #[inline(always)] diff --git a/src/frontend/src/catalog/mod.rs b/src/frontend/src/catalog/mod.rs index e497d0cd2ae4f..0dc1b96a46e39 100644 --- a/src/frontend/src/catalog/mod.rs +++ b/src/frontend/src/catalog/mod.rs @@ -25,6 +25,8 @@ pub(crate) mod schema_catalog; pub(crate) mod source_catalog; pub(crate) mod table_catalog; +pub use table_catalog::TableCatalog; + pub(crate) type SourceId = u32; pub(crate) type DatabaseId = u32; diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 0976a783d8718..f3d04792a8d44 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -16,6 +16,7 @@ use std::collections::{HashMap, HashSet}; use itertools::Itertools; use risingwave_common::catalog::{ColumnDesc, OrderedColumnDesc, TableDesc}; +use risingwave_common::util::compress::decompress_data; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::Table as ProstTable; @@ -52,8 +53,12 @@ pub struct TableCatalog { /// on this to derive an append-only stream plan pub appendonly: bool, - // owner of table + /// Owner of the table. pub owner: String, + + /// Mapping from vnode to parallel unit. Indicates data distribution and partition of the + /// table. + pub vnode_mapping: Option>, } impl TableCatalog { @@ -133,6 +138,7 @@ impl TableCatalog { .collect_vec(), appendonly: self.appendonly, owner: self.owner.clone(), + mapping: None, } } } @@ -173,6 +179,11 @@ impl From for TableCatalog { }) .collect(); + let vnode_mapping = decompress_data( + &tb.mapping.as_ref().unwrap().original_indices, + &tb.mapping.as_ref().unwrap().data, + ); + Self { id: id.into(), associated_source_id: associated_source_id.map(Into::into), @@ -192,6 +203,7 @@ impl From for TableCatalog { pks: tb.pk.iter().map(|x| *x as _).collect(), appendonly: tb.appendonly, owner: tb.owner, + vnode_mapping: Some(vnode_mapping), } } } @@ -207,9 +219,11 @@ mod tests { use risingwave_common::catalog::{ColumnDesc, ColumnId, OrderedColumnDesc, TableId}; use risingwave_common::test_prelude::*; use risingwave_common::types::*; + use risingwave_common::util::compress::compress_data; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::Table as ProstTable; + use risingwave_pb::common::ParallelUnitMapping; use risingwave_pb::plan_common::{ ColumnCatalog as ProstColumnCatalog, ColumnDesc as ProstColumnDesc, }; @@ -220,6 +234,8 @@ mod tests { #[test] fn test_into_table_catalog() { + let mapping = [1, 1, 2, 2, 3, 3, 4, 4].to_vec(); + let (original_indices, data) = compress_data(&mapping); let table: TableCatalog = ProstTable { is_index: false, index_on_id: 0, @@ -262,6 +278,11 @@ mod tests { .into(), appendonly: false, owner: risingwave_common::catalog::DEFAULT_SUPPER_USER.to_string(), + mapping: Some(ParallelUnitMapping { + table_id: 0, + original_indices, + data, + }), } .into(); @@ -310,6 +331,7 @@ mod tests { distribution_keys: vec![], appendonly: false, owner: risingwave_common::catalog::DEFAULT_SUPPER_USER.to_string(), + vnode_mapping: Some(mapping), } ); } diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 3adc289d389a8..85a5fb6c717d2 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -96,7 +96,7 @@ pub(crate) fn gen_create_table_plan( Ok((plan, source, table)) } -/// Generate a stream plan with `StreamSource` + `StreamMaterialize`, it ressembles a +/// Generate a stream plan with `StreamSource` + `StreamMaterialize`, it resembles a /// `CREATE MATERIALIZED VIEW AS SELECT * FROM `. pub(crate) fn gen_materialized_source_plan( context: OptimizerContextRef, diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 5d902a56de87c..70993c7ffbdc9 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -45,8 +45,12 @@ mod set; mod show; pub mod util; -pub(super) async fn handle(session: Arc, stmt: Statement) -> Result { - let context = OptimizerContext::new(session.clone()); +pub(super) async fn handle( + session: Arc, + stmt: Statement, + sql: &str, +) -> Result { + let context = OptimizerContext::new(session.clone(), Arc::from(sql)); match stmt { Statement::Explain { statement, verbose, .. diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index b0c8dc170d281..d6a74e7c5069c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -164,6 +164,7 @@ impl StreamMaterialize { distribution_keys: base.dist.dist_column_indices().to_vec(), appendonly: input.append_only(), owner: risingwave_common::catalog::DEFAULT_SUPPER_USER.to_string(), + vnode_mapping: None, }; Ok(Self { base, input, table }) diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 283542a89702a..445cb00fb9cba 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -62,6 +62,8 @@ pub struct OptimizerContext { pub session_ctx: Arc, // We use `AtomicI32` here because `Arc` implements `Send` only when `T: Send + Sync`. pub next_id: AtomicI32, + /// For debugging purposes, store the SQL string in Context + pub sql: Arc, } #[derive(Clone, Debug)] @@ -93,10 +95,11 @@ impl OptimizerContextRef { } impl OptimizerContext { - pub fn new(session_ctx: Arc) -> Self { + pub fn new(session_ctx: Arc, sql: Arc) -> Self { Self { session_ctx, next_id: AtomicI32::new(0), + sql, } } @@ -106,6 +109,7 @@ impl OptimizerContext { Self { session_ctx: Arc::new(SessionImpl::mock()), next_id: AtomicI32::new(0), + sql: Arc::from(""), } .into() } @@ -549,7 +553,7 @@ impl Session for SessionImpl { )); } let stmt = stmts.swap_remove(0); - let rsp = handle(self, stmt).await.map_err(|e| { + let rsp = handle(self, stmt, sql).await.map_err(|e| { tracing::error!("failed to handle sql:\n{}:\n{}", sql, e); e })?; @@ -571,7 +575,7 @@ impl Session for SessionImpl { return Ok(vec![]); } let stmt = stmts.swap_remove(0); - let rsp = infer(self, stmt).map_err(|e| { + let rsp = infer(self, stmt, sql).map_err(|e| { tracing::error!("failed to handle sql:\n{}:\n{}", sql, e); e })?; @@ -584,8 +588,8 @@ impl Session for SessionImpl { } /// Returns row description of the statement -fn infer(session: Arc, stmt: Statement) -> Result> { - let context = OptimizerContext::new(session); +fn infer(session: Arc, stmt: Statement, sql: &str) -> Result> { + let context = OptimizerContext::new(session, Arc::from(sql)); let session = context.session_ctx.clone(); let bound = { diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 281e32cf2f5bb..4a7f1c5124907 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -28,6 +28,7 @@ use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ Database as ProstDatabase, Schema as ProstSchema, Source as ProstSource, Table as ProstTable, }; +use risingwave_pb::common::ParallelUnitMapping; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_pb::user::{GrantPrivilege, UserInfo}; use risingwave_rpc_client::error::Result as RpcResult; @@ -91,7 +92,8 @@ impl LocalFrontend { /// Convert a sql (must be an `Query`) into an unoptimized batch plan. pub async fn to_batch_plan(&self, sql: impl Into) -> Result { - let statements = Parser::parse_sql(&sql.into()).unwrap(); + let raw_sql = &sql.into(); + let statements = Parser::parse_sql(raw_sql).unwrap(); let statement = statements.get(0).unwrap(); if let Statement::Query(query) = statement { let session = self.session_ref(); @@ -103,7 +105,7 @@ impl LocalFrontend { ); binder.bind(Statement::Query(query.clone()))? }; - Planner::new(OptimizerContext::new(session).into()) + Planner::new(OptimizerContext::new(session, Arc::from(raw_sql.as_str())).into()) .plan(bound) .unwrap() .gen_batch_query_plan() @@ -163,6 +165,11 @@ impl CatalogWriter for MockCatalogWriter { _graph: StreamFragmentGraph, ) -> Result<()> { table.id = self.gen_id(); + table.mapping = Some(ParallelUnitMapping { + table_id: table.id, + original_indices: [0, 10, 20].to_vec(), + data: [1, 2, 3].to_vec(), + }); self.catalog.write().create_table(&table); self.add_table_or_source_id(table.id, table.schema_id, table.database_id); Ok(()) diff --git a/src/frontend/test_runner/src/lib.rs b/src/frontend/test_runner/src/lib.rs index 9c62a42065c41..db0fa9b307ef1 100644 --- a/src/frontend/test_runner/src/lib.rs +++ b/src/frontend/test_runner/src/lib.rs @@ -248,7 +248,7 @@ impl TestCase { ) -> Result> { let statements = Parser::parse_sql(sql).unwrap(); for stmt in statements { - let context = OptimizerContext::new(session.clone()); + let context = OptimizerContext::new(session.clone(), Arc::from(sql)); match stmt.clone() { Statement::Query(_) | Statement::Insert { .. } diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 1f063703e904f..4dd2a2d95c65f 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -22,7 +22,7 @@ use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::util::epoch::INVALID_EPOCH; -use risingwave_hummock_sdk::HummockEpoch; +use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; use risingwave_pb::common::worker_node::State::Running; use risingwave_pb::common::WorkerType; use risingwave_pb::data::Barrier; @@ -324,9 +324,15 @@ where // We must ensure all epochs are committed in ascending order, because // the storage engine will query from new to old in the order in which // the L0 layer files are generated. see https://github.com/singularity-data/risingwave/issues/1251 - let synced_ssts = resps + let synced_ssts: Vec = resps .iter() .flat_map(|resp| resp.sycned_sstables.clone()) + .map(|grouped| { + ( + grouped.compaction_group_id, + grouped.sst.expect("field not None"), + ) + }) .collect_vec(); self.hummock_manager .commit_epoch(command_context.prev_epoch.0, synced_ssts) diff --git a/src/meta/src/cluster/mod.rs b/src/meta/src/cluster/mod.rs index 34e2f50f47b65..40b4b39e78c9c 100644 --- a/src/meta/src/cluster/mod.rs +++ b/src/meta/src/cluster/mod.rs @@ -23,6 +23,7 @@ use std::time::{Duration, SystemTime}; use itertools::Itertools; use risingwave_common::error::{internal_error, ErrorCode, Result}; use risingwave_common::try_match_expand; +use risingwave_common::types::ParallelUnitId; use risingwave_pb::common::worker_node::State; use risingwave_pb::common::{HostAddress, ParallelUnit, ParallelUnitType, WorkerNode, WorkerType}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -35,7 +36,6 @@ use crate::model::{MetadataModel, Worker, INVALID_EXPIRE_AT}; use crate::storage::MetaStore; pub type WorkerId = u32; -pub type ParallelUnitId = u32; pub type WorkerLocations = HashMap; pub type ClusterManagerRef = Arc>; diff --git a/src/meta/src/hummock/compaction/compaction_config.rs b/src/meta/src/hummock/compaction/compaction_config.rs index a39ad8f800b5d..590d469a137ff 100644 --- a/src/meta/src/hummock/compaction/compaction_config.rs +++ b/src/meta/src/hummock/compaction/compaction_config.rs @@ -38,7 +38,7 @@ impl CompactionConfigBuilder { min_compaction_bytes: DEFAULT_MIN_COMPACTION_BYTES, level0_tigger_file_numer: DEFAULT_TIER_COMPACT_TRIGGER_NUMBER * 2, level0_tier_compact_file_number: DEFAULT_TIER_COMPACT_TRIGGER_NUMBER, - compaction_mode: CompactionMode::ConsistentHash as i32, + compaction_mode: CompactionMode::Range as i32, }, } } diff --git a/src/meta/src/hummock/compaction/level_selector.rs b/src/meta/src/hummock/compaction/level_selector.rs index 9392c77466db7..75ed339f58492 100644 --- a/src/meta/src/hummock/compaction/level_selector.rs +++ b/src/meta/src/hummock/compaction/level_selector.rs @@ -23,12 +23,15 @@ use risingwave_hummock_sdk::HummockCompactionTaskId; use risingwave_pb::hummock::{CompactionConfig, Level}; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; -use crate::hummock::compaction::compaction_picker::{CompactionPicker, MinOverlappingPicker}; +use crate::hummock::compaction::manual_compaction_picker::ManualCompactionPicker; +use crate::hummock::compaction::min_overlap_compaction_picker::MinOverlappingPicker; use crate::hummock::compaction::overlap_strategy::OverlapStrategy; use crate::hummock::compaction::tier_compaction_picker::{ LevelCompactionPicker, TierCompactionPicker, }; -use crate::hummock::compaction::{create_overlap_strategy, SearchResult}; +use crate::hummock::compaction::{ + create_overlap_strategy, CompactionPicker, ManualCompactionOption, SearchResult, +}; use crate::hummock::level_handler::LevelHandler; const SCORE_BASE: u64 = 100; @@ -43,6 +46,14 @@ pub trait LevelSelector: Sync + Send { level_handlers: &mut [LevelHandler], ) -> Option; + fn manual_pick_compaction( + &self, + task_id: HummockCompactionTaskId, + levels: &[Level], + level_handlers: &mut [LevelHandler], + option: ManualCompactionOption, + ) -> Option; + fn name(&self) -> &'static str; } @@ -239,6 +250,30 @@ impl LevelSelector for DynamicLevelSelector { None } + fn manual_pick_compaction( + &self, + task_id: HummockCompactionTaskId, + levels: &[Level], + level_handlers: &mut [LevelHandler], + option: ManualCompactionOption, + ) -> Option { + let ctx = self.get_priority_levels(levels, level_handlers); + let target_level = if option.level == 0 { + ctx.base_level + } else { + option.level + 1 + }; + + let picker = ManualCompactionPicker::new( + task_id, + self.overlap_strategy.clone(), + option, + target_level, + ); + + picker.pick_compaction(levels, level_handlers) + } + fn name(&self) -> &'static str { "DynamicLevelSelector" } diff --git a/src/meta/src/hummock/compaction/manual_compaction_picker.rs b/src/meta/src/hummock/compaction/manual_compaction_picker.rs new file mode 100644 index 0000000000000..f93fbdd3b6c15 --- /dev/null +++ b/src/meta/src/hummock/compaction/manual_compaction_picker.rs @@ -0,0 +1,321 @@ +// Copyright 2022 Singularity Data +// +// 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::sync::Arc; + +use itertools::Itertools; +use risingwave_pb::hummock::{Level, SstableInfo}; + +use super::overlap_strategy::OverlapInfo; +use super::CompactionPicker; +use crate::hummock::compaction::overlap_strategy::{OverlapStrategy, RangeOverlapInfo}; +use crate::hummock::compaction::{ManualCompactionOption, SearchResult}; +use crate::hummock::level_handler::LevelHandler; + +pub struct ManualCompactionPicker { + compact_task_id: u64, + overlap_strategy: Arc, + option: ManualCompactionOption, + target_level: usize, +} + +impl ManualCompactionPicker { + pub fn new( + compact_task_id: u64, + overlap_strategy: Arc, + option: ManualCompactionOption, + target_level: usize, + ) -> Self { + Self { + compact_task_id, + overlap_strategy, + option, + target_level, + } + } +} + +impl CompactionPicker for ManualCompactionPicker { + fn pick_compaction( + &self, + levels: &[Level], + level_handlers: &mut [LevelHandler], + ) -> Option { + let level = self.option.level; + let target_level = self.target_level; + + let mut select_input_ssts = vec![]; + let mut tmp_sst_info = SstableInfo::default(); + let mut range_overlap_info = RangeOverlapInfo::default(); + tmp_sst_info.key_range = Some(self.option.key_range.clone()); + range_overlap_info.update(&tmp_sst_info); + + let level_table_infos: Vec = levels[level] + .table_infos + .iter() + .filter(|sst_info| range_overlap_info.check_overlap(sst_info)) + .filter(|sst_info| { + if self.option.internal_table_id.is_empty() { + return true; + } + + // to collect internal_table_id from sst_info + let table_id_in_sst: Vec = + sst_info.get_table_ids().iter().cloned().collect_vec(); + + // to filter sst_file by table_id + for table_id in &table_id_in_sst { + if self.option.internal_table_id.contains(table_id) { + return true; + } + } + + false + }) + .cloned() + .collect(); + + for table in &level_table_infos { + if level_handlers[level].is_pending_compact(&table.id) { + continue; + } + + let overlap_files = self + .overlap_strategy + .check_base_level_overlap(&[table.clone()], &levels[target_level].table_infos); + + if overlap_files + .iter() + .any(|table| level_handlers[target_level].is_pending_compact(&table.id)) + { + continue; + } + + select_input_ssts.push(table.clone()); + } + + if select_input_ssts.is_empty() { + return None; + } + + let target_input_ssts = self + .overlap_strategy + .check_base_level_overlap(&select_input_ssts, &levels[target_level].table_infos); + + if target_input_ssts + .iter() + .any(|table| level_handlers[level].is_pending_compact(&table.id)) + { + return None; + } + + level_handlers[level].add_pending_task(self.compact_task_id, &select_input_ssts); + if !target_input_ssts.is_empty() { + level_handlers[target_level].add_pending_task(self.compact_task_id, &target_input_ssts); + } + + Some(SearchResult { + select_level: Level { + level_idx: level as u32, + level_type: levels[level].level_type, + total_file_size: select_input_ssts.iter().map(|table| table.file_size).sum(), + table_infos: select_input_ssts, + }, + target_level: Level { + level_idx: target_level as u32, + level_type: levels[target_level].level_type, + total_file_size: target_input_ssts.iter().map(|table| table.file_size).sum(), + table_infos: target_input_ssts, + }, + split_ranges: vec![], + }) + } +} + +#[cfg(test)] +pub mod tests { + use std::collections::HashSet; + + pub use risingwave_pb::hummock::{KeyRange, LevelType}; + + use super::*; + use crate::hummock::compaction::overlap_strategy::RangeOverlapStrategy; + use crate::hummock::compaction::tier_compaction_picker::tests::generate_table; + use crate::hummock::test_utils::iterator_test_key_of_epoch; + + #[test] + fn test_manaul_compaction_picker() { + let mut levels = vec![ + Level { + level_idx: 0, + level_type: LevelType::Overlapping as i32, + table_infos: vec![], + total_file_size: 0, + }, + Level { + level_idx: 1, + level_type: LevelType::Nonoverlapping as i32, + table_infos: vec![ + generate_table(0, 1, 0, 100, 1), + generate_table(1, 1, 101, 200, 1), + generate_table(2, 1, 222, 300, 1), + ], + total_file_size: 0, + }, + Level { + level_idx: 2, + level_type: LevelType::Nonoverlapping as i32, + table_infos: vec![ + generate_table(4, 1, 0, 100, 1), + generate_table(5, 1, 101, 150, 1), + generate_table(6, 1, 151, 201, 1), + generate_table(7, 1, 501, 800, 1), + generate_table(8, 2, 301, 400, 1), + ], + total_file_size: 0, + }, + ]; + let mut levels_handler = vec![ + LevelHandler::new(0), + LevelHandler::new(1), + LevelHandler::new(2), + ]; + + let clean_task_state = |level_handler: &mut LevelHandler| { + for pending_task_id in &level_handler.pending_tasks_ids() { + level_handler.remove_task(*pending_task_id); + } + }; + + { + // test key_range option + let option = ManualCompactionOption { + level: 1, + key_range: KeyRange { + left: iterator_test_key_of_epoch(1, 0, 1), + right: iterator_test_key_of_epoch(1, 201, 1), + inf: false, + }, + ..Default::default() + }; + + let target_level = option.level + 1; + let picker = ManualCompactionPicker::new( + 0, + Arc::new(RangeOverlapStrategy::default()), + option, + target_level, + ); + let result = picker + .pick_compaction(&levels, &mut levels_handler) + .unwrap(); + + assert_eq!(2, result.select_level.table_infos.len()); + assert_eq!(3, result.target_level.table_infos.len()); + } + + { + clean_task_state(&mut levels_handler[1]); + clean_task_state(&mut levels_handler[2]); + + // test all key range + let option = ManualCompactionOption::default(); + let target_level = option.level + 1; + let picker = ManualCompactionPicker::new( + 0, + Arc::new(RangeOverlapStrategy::default()), + option, + target_level, + ); + let result = picker + .pick_compaction(&levels, &mut levels_handler) + .unwrap(); + + assert_eq!(3, result.select_level.table_infos.len()); + assert_eq!(3, result.target_level.table_infos.len()); + } + + { + clean_task_state(&mut levels_handler[1]); + clean_task_state(&mut levels_handler[2]); + + let level_table_info = &mut levels[1].table_infos; + let table_info_1 = &mut level_table_info[1]; + table_info_1.table_ids.resize(2, 0); + table_info_1.table_ids[0] = 1; + table_info_1.table_ids[1] = 2; + + // test internal_table_id + let option = ManualCompactionOption { + level: 1, + internal_table_id: HashSet::from([2]), + ..Default::default() + }; + + let target_level = option.level + 1; + let picker = ManualCompactionPicker::new( + 0, + Arc::new(RangeOverlapStrategy::default()), + option, + target_level, + ); + + let result = picker + .pick_compaction(&levels, &mut levels_handler) + .unwrap(); + + assert_eq!(1, result.select_level.table_infos.len()); + assert_eq!(2, result.target_level.table_infos.len()); + } + + { + clean_task_state(&mut levels_handler[1]); + clean_task_state(&mut levels_handler[2]); + + // include all table_info + let level_table_info = &mut levels[1].table_infos; + for table_info in level_table_info { + table_info.table_ids.resize(2, 0); + table_info.table_ids[0] = 1; + table_info.table_ids[1] = 2; + } + + // test key range filter first + let option = ManualCompactionOption { + level: 1, + key_range: KeyRange { + left: iterator_test_key_of_epoch(1, 101, 1), + right: iterator_test_key_of_epoch(1, 199, 1), + inf: false, + }, + internal_table_id: HashSet::from([2]), + }; + + let target_level = option.level + 1; + let picker = ManualCompactionPicker::new( + 0, + Arc::new(RangeOverlapStrategy::default()), + option, + target_level, + ); + + let result = picker + .pick_compaction(&levels, &mut levels_handler) + .unwrap(); + + assert_eq!(1, result.select_level.table_infos.len()); + assert_eq!(2, result.target_level.table_infos.len()); + } + } +} diff --git a/src/meta/src/hummock/compaction/compaction_picker.rs b/src/meta/src/hummock/compaction/min_overlap_compaction_picker.rs similarity index 97% rename from src/meta/src/hummock/compaction/compaction_picker.rs rename to src/meta/src/hummock/compaction/min_overlap_compaction_picker.rs index 26b686e1b18f9..77cb811b9f745 100644 --- a/src/meta/src/hummock/compaction/compaction_picker.rs +++ b/src/meta/src/hummock/compaction/min_overlap_compaction_picker.rs @@ -17,18 +17,11 @@ use std::sync::Arc; use risingwave_pb::hummock::{Level, SstableInfo}; +use super::CompactionPicker; use crate::hummock::compaction::overlap_strategy::OverlapStrategy; use crate::hummock::compaction::SearchResult; use crate::hummock::level_handler::LevelHandler; -pub trait CompactionPicker { - fn pick_compaction( - &self, - levels: &[Level], - level_handlers: &mut [LevelHandler], - ) -> Option; -} - pub struct MinOverlappingPicker { compact_task_id: u64, overlap_strategy: Arc, @@ -175,14 +168,14 @@ impl CompactionPicker for MinOverlappingPicker { select_level: Level { level_idx: self.level as u32, level_type: levels[self.level].level_type, + total_file_size: select_input_ssts.iter().map(|table| table.file_size).sum(), table_infos: select_input_ssts, - total_file_size: 0, }, target_level: Level { level_idx: target_level as u32, level_type: levels[target_level].level_type, + total_file_size: target_input_ssts.iter().map(|table| table.file_size).sum(), table_infos: target_input_ssts, - total_file_size: 0, }, split_ranges: vec![], }) @@ -191,7 +184,7 @@ impl CompactionPicker for MinOverlappingPicker { #[cfg(test)] pub mod tests { - use risingwave_pb::hummock::LevelType; + pub use risingwave_pb::hummock::{KeyRange, LevelType}; use super::*; use crate::hummock::compaction::overlap_strategy::RangeOverlapStrategy; diff --git a/src/meta/src/hummock/compaction/mod.rs b/src/meta/src/hummock/compaction/mod.rs index d829f06afb55b..ec78c578d3262 100644 --- a/src/meta/src/hummock/compaction/mod.rs +++ b/src/meta/src/hummock/compaction/mod.rs @@ -13,16 +13,17 @@ // limitations under the License. pub mod compaction_config; -mod compaction_picker; mod level_selector; +mod manual_compaction_picker; +mod min_overlap_compaction_picker; mod overlap_strategy; mod prost_type; mod tier_compaction_picker; - use std::collections::HashSet; use std::fmt::{Debug, Formatter}; use std::sync::Arc; +use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; use risingwave_hummock_sdk::{CompactionGroupId, HummockCompactionTaskId, HummockEpoch}; use risingwave_pb::hummock::compaction_config::CompactionMode; @@ -70,6 +71,7 @@ impl Clone for CompactStatus { } } +#[derive(Debug, Default)] pub struct SearchResult { select_level: Level, target_level: Level, @@ -106,15 +108,24 @@ impl CompactStatus { levels: &[Level], task_id: HummockCompactionTaskId, compaction_group_id: CompactionGroupId, + manual_compaction_option: Option, ) -> 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 = match self.pick_compaction(levels, task_id) { - Some(ret) => ret, - None => return None, - }; + let ret; + if let Some(manual_compaction_option) = manual_compaction_option { + ret = match self.manual_pick_compaction(levels, task_id, manual_compaction_option) { + Some(ret) => ret, + None => return None, + }; + } else { + ret = match self.pick_compaction(levels, task_id) { + Some(ret) => ret, + None => return None, + }; + } let select_level_id = ret.select_level.level_idx; let target_level_id = ret.target_level.level_idx; @@ -152,6 +163,22 @@ impl CompactStatus { .pick_compaction(task_id, levels, &mut self.level_handlers) } + fn manual_pick_compaction( + &mut self, + levels: &[Level], + task_id: HummockCompactionTaskId, + manual_compaction_option: ManualCompactionOption, + ) -> Option { + // manual_compaction no need to select level + // level determined by option + self.compaction_selector.manual_pick_compaction( + task_id, + levels, + &mut self.level_handlers, + manual_compaction_option, + ) + } + /// Declares a task is either finished or canceled. pub fn report_compact_task(&mut self, compact_task: &CompactTask) { for level in &compact_task.input_ssts { @@ -185,14 +212,16 @@ impl CompactStatus { removed_table.insert(table.id); } } + let new_version_levels = + new_version.get_compaction_group_levels_mut(compact_task.compaction_group_id); if compact_task.target_level == 0 { assert_eq!(compact_task.input_ssts[0].level_idx, 0); let mut new_table_infos = vec![]; let mut find_remove_position = false; let mut new_total_file_size = 0; - for (idx, table) in new_version.levels[0].table_infos.iter().enumerate() { + for (idx, table) in new_version_levels[0].table_infos.iter().enumerate() { if !removed_table.contains(&table.id) { - new_table_infos.push(new_version.levels[0].table_infos[idx].clone()); + new_table_infos.push(new_version_levels[0].table_infos[idx].clone()); new_total_file_size += table.file_size; } else if !find_remove_position { new_total_file_size += compact_task @@ -204,29 +233,29 @@ impl CompactStatus { find_remove_position = true; } } - new_version.levels[compact_task.target_level as usize].table_infos = new_table_infos; - new_version.levels[compact_task.target_level as usize].total_file_size = + new_version_levels[compact_task.target_level as usize].table_infos = new_table_infos; + new_version_levels[compact_task.target_level as usize].total_file_size = new_total_file_size; } else { for input_level in &compact_task.input_ssts { - new_version.levels[input_level.level_idx as usize].total_file_size -= input_level + new_version_levels[input_level.level_idx as usize].total_file_size -= input_level .table_infos .iter() .map(|sst| sst.file_size) .sum::(); - new_version.levels[input_level.level_idx as usize] + new_version_levels[input_level.level_idx as usize] .table_infos .retain(|sst| !removed_table.contains(&sst.id)); } - new_version.levels[compact_task.target_level as usize].total_file_size += compact_task + new_version_levels[compact_task.target_level as usize].total_file_size += compact_task .sorted_output_ssts .iter() .map(|sst| sst.file_size) .sum::(); - new_version.levels[compact_task.target_level as usize] + new_version_levels[compact_task.target_level as usize] .table_infos .extend(compact_task.sorted_output_ssts.clone()); - new_version.levels[compact_task.target_level as usize] + new_version_levels[compact_task.target_level as usize] .table_infos .sort_by(|sst1, sst2| { let a = sst1.key_range.as_ref().unwrap(); @@ -241,3 +270,32 @@ impl CompactStatus { self.compaction_group_id } } + +#[derive(Clone, Debug)] +pub struct ManualCompactionOption { + pub key_range: KeyRange, + pub internal_table_id: HashSet, + pub level: usize, +} + +impl Default for ManualCompactionOption { + fn default() -> Self { + Self { + key_range: KeyRange { + left: vec![], + right: vec![], + inf: true, + }, + internal_table_id: HashSet::default(), + level: 1, + } + } +} + +pub trait CompactionPicker { + fn pick_compaction( + &self, + levels: &[Level], + level_handlers: &mut [LevelHandler], + ) -> Option; +} diff --git a/src/meta/src/hummock/compaction/overlap_strategy.rs b/src/meta/src/hummock/compaction/overlap_strategy.rs index de8f9a3907754..667d38a62018a 100644 --- a/src/meta/src/hummock/compaction/overlap_strategy.rs +++ b/src/meta/src/hummock/compaction/overlap_strategy.rs @@ -165,16 +165,16 @@ fn check_key_vnode_overlap(info: &SstableInfo, table: &SstableInfo) -> bool { { return false; } - let text_len = info.get_vnode_bitmaps().len(); - let other_len = table.get_vnode_bitmaps().len(); + let text_len = info.get_table_ids().len(); + let other_len = table.get_table_ids().len(); if text_len == 0 || other_len == 0 { return true; } let (mut i, mut j) = (0, 0); while i < text_len && j < other_len { - let x = &info.get_vnode_bitmaps()[i]; - let y = &table.get_vnode_bitmaps()[j]; - match x.get_table_id().cmp(&y.get_table_id()) { + let x = &info.get_table_ids()[i]; + let y = &table.get_table_ids()[j]; + match x.cmp(y) { Ordering::Less => { i += 1; } @@ -182,15 +182,9 @@ fn check_key_vnode_overlap(info: &SstableInfo, table: &SstableInfo) -> bool { j += 1; } Ordering::Equal => { - let maplen = x.get_bitmap().len(); - assert_eq!(maplen, y.get_bitmap().len()); - for bitmap_idx in 0..maplen as usize { - if (x.get_bitmap()[bitmap_idx] & y.get_bitmap()[bitmap_idx]) != 0 { - return true; - } - } - i += 1; - j += 1; + return true; + // i += 1; + // j += 1; } } } diff --git a/src/meta/src/hummock/compaction/tier_compaction_picker.rs b/src/meta/src/hummock/compaction/tier_compaction_picker.rs index 1187f67a9f708..71b66d6e1bddd 100644 --- a/src/meta/src/hummock/compaction/tier_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/tier_compaction_picker.rs @@ -21,8 +21,8 @@ use risingwave_hummock_sdk::HummockEpoch; use risingwave_pb::hummock::{CompactionConfig, KeyRange, Level, LevelType, SstableInfo}; use super::SearchResult; -use crate::hummock::compaction::compaction_picker::CompactionPicker; use crate::hummock::compaction::overlap_strategy::OverlapStrategy; +use crate::hummock::compaction::CompactionPicker; use crate::hummock::level_handler::LevelHandler; const MIN_COMPACTION_BYTES: u64 = 2 * 1024 * 1024; // 1MB @@ -199,15 +199,20 @@ impl CompactionPicker for LevelCompactionPicker { select_level: Level { level_idx: select_level as u32, level_type: LevelType::Overlapping as i32, + total_file_size: select_level_inputs + .iter() + .map(|table| table.file_size) + .sum(), table_infos: select_level_inputs, - // no use - total_file_size: 0, }, target_level: Level { level_idx: target_level as u32, level_type: LevelType::Nonoverlapping as i32, + total_file_size: target_level_inputs + .iter() + .map(|table| table.file_size) + .sum(), table_infos: target_level_inputs, - total_file_size: 0, }, split_ranges: splits, }) @@ -379,7 +384,7 @@ pub mod tests { inf: false, }), file_size: (right - left + 1) as u64, - vnode_bitmaps: vec![], + table_ids: vec![], unit_id: u64::MAX, } } diff --git a/src/meta/src/hummock/compaction_group/manager.rs b/src/meta/src/hummock/compaction_group/manager.rs index 9ebd93091388e..4a6cdd144c0a9 100644 --- a/src/meta/src/hummock/compaction_group/manager.rs +++ b/src/meta/src/hummock/compaction_group/manager.rs @@ -24,13 +24,18 @@ use tokio::sync::RwLock; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; use crate::hummock::compaction_group::CompactionGroup; use crate::hummock::error::{Error, Result}; -use crate::manager::MetaSrvEnv; +use crate::manager::{MetaSrvEnv, SourceId}; use crate::model::{MetadataModel, TableFragments, ValTransaction, VarTransaction}; use crate::storage::{MetaStore, Transaction}; pub type CompactionGroupManagerRef = Arc>; /// `CompactionGroupManager` manages `CompactionGroup`'s members. +/// +/// Note that all hummock state store user should register to `CompactionGroupManager`. It includes: +/// - Materialized View via `register_table_fragments`. +/// - Materialized Source via `register_table_fragments`. +/// - Source via `register_source`. pub struct CompactionGroupManager { env: MetaSrvEnv, inner: RwLock, @@ -73,9 +78,7 @@ impl CompactionGroupManager { /// Registers `table_fragments` to compaction groups. pub async fn register_table_fragments(&self, table_fragments: &TableFragments) -> Result<()> { let mut pairs = vec![]; - // MV or source - // existing_table_ids include the table_ref_id (source and materialized_view) + - // internal_table_id (stateful executor) + // materialized_view or materialized_source pairs.push(( Prefix::from(table_fragments.table_id().table_id), // TODO: before compaction group write path is finished, all SSTs belongs to @@ -114,10 +117,33 @@ impl CompactionGroupManager { .await } - /// Unregisters members that doesn't belong to `table_fragments_list` from compaction groups + pub async fn register_source(&self, source_id: u32) -> Result<()> { + self.inner + .write() + .await + .register( + &[( + source_id.into(), + StaticCompactionGroupId::StateDefault.into(), + )], + self.env.meta_store(), + ) + .await + } + + pub async fn unregister_source(&self, source_id: u32) -> Result<()> { + self.inner + .write() + .await + .unregister(&[source_id.into()], self.env.meta_store()) + .await + } + + /// Unregisters stale members pub async fn purge_stale_members( &self, - table_fragments_list: &[&TableFragments], + table_fragments_list: &[TableFragments], + source_ids: &[SourceId], ) -> Result<()> { let mut guard = self.inner.write().await; let registered_members = guard @@ -135,6 +161,7 @@ impl CompactionGroupManager { .chain(std::iter::once(table_fragments.table_id().table_id)) .collect_vec() }) + .chain(source_ids.iter().cloned()) .collect_vec(); let to_unregister = registered_members .into_iter() @@ -146,7 +173,7 @@ impl CompactionGroupManager { .await } - pub async fn internal_table_ids_by_compation_group_id( + pub async fn internal_table_ids_by_compaction_group_id( &self, compaction_group_id: u64, ) -> Result> { @@ -359,8 +386,11 @@ mod tests { TableFragments::new(TableId::new(10), Default::default(), [11, 12, 13].into()); let table_fragment_2 = TableFragments::new(TableId::new(20), Default::default(), [21, 22, 23].into()); + let source_1 = 100; + let source_2 = 200; + let source_3 = 300; - // Test register_stream_nodes + // Test register_table_fragments let registered_number = || async { compaction_group_manager .compaction_groups() @@ -381,23 +411,59 @@ mod tests { .unwrap(); assert_eq!(registered_number().await, 8); - // Test unregister_stream_nodes + // Test unregister_table_fragments compaction_group_manager .unregister_table_fragments(&table_fragment_1) .await .unwrap(); assert_eq!(registered_number().await, 4); - // Test purge_stale_members + // Test purge_stale_members: table fragments compaction_group_manager - .purge_stale_members(&[&table_fragment_2]) + .purge_stale_members(&[table_fragment_2], &[]) .await .unwrap(); assert_eq!(registered_number().await, 4); compaction_group_manager - .purge_stale_members(&[]) + .purge_stale_members(&[], &[]) .await .unwrap(); assert_eq!(registered_number().await, 0); + + // Test register_source + compaction_group_manager + .register_source(source_1) + .await + .unwrap(); + assert_eq!(registered_number().await, 1); + compaction_group_manager + .register_source(source_2) + .await + .unwrap(); + assert_eq!(registered_number().await, 2); + compaction_group_manager + .register_source(source_2) + .await + .unwrap(); + assert_eq!(registered_number().await, 2); + compaction_group_manager + .register_source(source_3) + .await + .unwrap(); + assert_eq!(registered_number().await, 3); + + // Test unregister_source + compaction_group_manager + .unregister_source(source_2) + .await + .unwrap(); + assert_eq!(registered_number().await, 2); + + // Test purge_stale_members: source + compaction_group_manager + .purge_stale_members(&[], &[source_3]) + .await + .unwrap(); + assert_eq!(registered_number().await, 1); } } diff --git a/src/meta/src/hummock/compactor_manager.rs b/src/meta/src/hummock/compactor_manager.rs index b6b410ad8ba30..4f8fc11dc110c 100644 --- a/src/meta/src/hummock/compactor_manager.rs +++ b/src/meta/src/hummock/compactor_manager.rs @@ -147,7 +147,9 @@ mod tests { use risingwave_pb::hummock::CompactTask; use tokio::sync::mpsc::error::TryRecvError; - use crate::hummock::test_utils::{generate_test_tables, setup_compute_env}; + use crate::hummock::test_utils::{ + generate_test_tables, setup_compute_env, to_local_sstable_info, + }; use crate::hummock::{CompactorManager, HummockManager}; use crate::storage::MetaStore; @@ -163,7 +165,7 @@ mod tests { vec![hummock_manager_ref.get_new_table_id().await.unwrap()], ); hummock_manager_ref - .commit_epoch(epoch, original_tables) + .commit_epoch(epoch, to_local_sstable_info(&original_tables)) .await .unwrap(); } @@ -179,7 +181,7 @@ mod tests { is_target_ultimate_and_leveling: false, task_status: false, vnode_mappings: vec![], - compaction_group_id: StaticCompactionGroupId::SharedBuffer.into(), + compaction_group_id: StaticCompactionGroupId::StateDefault.into(), existing_table_ids: vec![], } } diff --git a/src/meta/src/hummock/hummock_manager.rs b/src/meta/src/hummock/hummock_manager.rs index 88050164fe96e..09874eff5567d 100644 --- a/src/meta/src/hummock/hummock_manager.rs +++ b/src/meta/src/hummock/hummock_manager.rs @@ -20,15 +20,15 @@ use std::time::{Duration, Instant}; use itertools::Itertools; use prost::Message; -use risingwave_common::util::compress::compress_data; use risingwave_common::util::epoch::INVALID_EPOCH; use risingwave_hummock_sdk::compact::compact_task_to_string; +use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::{ get_remote_sst_id, CompactionGroupId, HummockCompactionTaskId, HummockContextId, HummockEpoch, - HummockRefCount, HummockSSTableId, HummockVersionId, + HummockRefCount, HummockSSTableId, HummockVersionId, LocalSstableInfo, }; -use risingwave_pb::common::ParallelUnitMapping; +use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::{ CompactTask, CompactTaskAssignment, HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, HummockStaleSstables, HummockVersion, Level, LevelType, SstableIdInfo, @@ -39,7 +39,7 @@ use risingwave_pb::meta::MetaLeaderInfo; use tokio::sync::RwLock; use crate::cluster::{ClusterManagerRef, META_NODE_ID}; -use crate::hummock::compaction::CompactStatus; +use crate::hummock::compaction::{CompactStatus, ManualCompactionOption}; use crate::hummock::compaction_group::manager::CompactionGroupManagerRef; use crate::hummock::compaction_scheduler::CompactionRequestChannelRef; use crate::hummock::error::{Error, Result}; @@ -229,31 +229,29 @@ where if versioning_guard.hummock_versions.is_empty() { let mut init_version = HummockVersion { id: versioning_guard.current_version_id.id(), - levels: vec![Level { - level_idx: 0, - level_type: LevelType::Overlapping as i32, - table_infos: vec![], - total_file_size: 0, - }], + levels: Default::default(), max_committed_epoch: INVALID_EPOCH, safe_epoch: INVALID_EPOCH, }; - // TODO #2065: Initialize independent levels via corresponding compaction group' config. - // Currently all SSTs belongs to `StateDefault`. - let max_level = self - .compaction_group_manager - .compaction_group(StaticCompactionGroupId::StateDefault.into()) - .await - .unwrap() - .compaction_config() - .max_level; - for l in 0..max_level { - init_version.levels.push(Level { - level_idx: (l + 1) as u32, - level_type: LevelType::Nonoverlapping as i32, + // Initialize independent levels via corresponding compaction group' config. + for compaction_group in self.compaction_group_manager.compaction_groups().await { + let mut levels = vec![Level { + level_idx: 0u32, + level_type: LevelType::Overlapping as i32, table_infos: vec![], total_file_size: 0, - }); + }]; + for l in 0..compaction_group.compaction_config().max_level { + levels.push(Level { + level_idx: (l + 1) as u32, + level_type: LevelType::Nonoverlapping as i32, + table_infos: vec![], + total_file_size: 0, + }); + } + init_version + .levels + .insert(compaction_group.group_id(), Levels { levels }); } init_version.insert(self.env.meta_store()).await?; versioning_guard @@ -553,9 +551,10 @@ where Ok(()) } - pub async fn get_compact_task( + pub async fn get_compact_task_impl( &self, compaction_group_id: CompactionGroupId, + manual_compaction_option: Option, ) -> Result> { // TODO #2065: Remove this line after split levels by compaction group. // All SSTs belongs to `StateDefault` currently. @@ -588,9 +587,10 @@ where ); let current_version = self.versioning.read().await.current_version(); let compact_task = compact_status.get_compact_task( - ¤t_version.levels, + current_version.get_compaction_group_levels(compaction_group_id), task_id as HummockCompactionTaskId, compaction_group_id, + manual_compaction_option, ); let ret = match compact_task { @@ -598,7 +598,7 @@ where Some(mut compact_task) => { let existing_table_ids_from_meta = self .compaction_group_manager - .internal_table_ids_by_compation_group_id(compaction_group_id) + .internal_table_ids_by_compaction_group_id(compaction_group_id) .await?; compact_task.watermark = { @@ -624,34 +624,12 @@ where level .table_infos .iter() - .flat_map(|sst_info| { - sst_info.vnode_bitmaps.iter().map(|bitmap| bitmap.table_id) - }) + .flat_map(|sst_info| sst_info.table_ids.iter().cloned()) .collect_vec() }) .collect::>(); - if compact_task.target_level != 0 { - compact_task.vnode_mappings.reserve_exact(table_ids.len()); - } - for table_id in table_ids { - if compact_task.target_level != 0 { - if let Some(vnode_mapping) = self - .env - .hash_mapping_manager() - .get_table_hash_mapping(&table_id) - { - let (original_indices, compressed_data) = compress_data(&vnode_mapping); - let compressed_mapping = ParallelUnitMapping { - table_id, - original_indices, - data: compressed_data, - }; - compact_task.vnode_mappings.push(compressed_mapping); - } - } - // to found exist table_id from if existing_table_ids_from_meta.contains(&table_id) { compact_task.existing_table_ids.push(table_id); @@ -660,10 +638,11 @@ where commit_multi_var!(self, None, compact_status)?; tracing::trace!( - "pick up {} tables in level {} to compact, The number of total tables is {}. cost time: {:?}", + "For compaction group {}: pick up {} tables in level {} to compact, The number of total tables is {}. cost time: {:?}", + compaction_group_id, compact_task.input_ssts[0].table_infos.len(), compact_task.input_ssts[0].level_idx, - current_version.levels[compact_task.input_ssts[0].level_idx as usize] + current_version.get_compaction_group_levels(compaction_group_id)[compact_task.input_ssts[0].level_idx as usize] .table_infos .len(), start_time.elapsed() @@ -681,6 +660,22 @@ where ret } + pub async fn get_compact_task( + &self, + compaction_group_id: CompactionGroupId, + ) -> Result> { + self.get_compact_task_impl(compaction_group_id, None).await + } + + pub async fn manual_get_compact_task( + &self, + compaction_group_id: CompactionGroupId, + manual_compaction_option: ManualCompactionOption, + ) -> Result> { + self.get_compact_task_impl(compaction_group_id, Some(manual_compaction_option)) + .await + } + /// Assigns a compaction task to a compactor pub async fn assign_compaction_task>( &self, @@ -837,8 +832,11 @@ where pub async fn commit_epoch( &self, epoch: HummockEpoch, - sstables: Vec, + sstables: Vec, ) -> Result<()> { + // TODO #2065: add SSTs to corresponding compaction groups' levels. + let sstables = sstables.into_iter().map(|(_, sst)| sst).collect_vec(); + let mut versioning_guard = self.versioning.write().await; let old_version = versioning_guard.current_version(); let versioning = versioning_guard.deref_mut(); @@ -890,8 +888,9 @@ where } // Create a new_version, possibly merely to bump up the version id and max_committed_epoch. + // TODO #2065: use correct compaction group id let version_first_level = new_hummock_version - .levels + .get_compaction_group_levels_mut(StaticCompactionGroupId::StateDefault.into()) .first_mut() .expect("Expect at least one level"); assert_eq!(version_first_level.level_idx, 0); @@ -1200,6 +1199,7 @@ where versioning .levels .iter() + .flat_map(|(_, l)| &l.levels) .flat_map(|level| { level.table_infos.iter().map(|table_info| { versioning_guard @@ -1360,6 +1360,7 @@ where pub async fn trigger_manual_compaction( &self, compaction_group: CompactionGroupId, + manual_compaction_option: ManualCompactionOption, ) -> Result<()> { let start_time = Instant::now(); @@ -1377,8 +1378,10 @@ where Some(compactor) => compactor, }; - // 2. compact_task - let compact_task = self.get_compact_task(compaction_group).await; + // 2. manual_get_compact_task + let compact_task = self + .manual_get_compact_task(compaction_group, manual_compaction_option) + .await; let compact_task = match compact_task { Ok(Some(compact_task)) => compact_task, Ok(None) => { diff --git a/src/meta/src/hummock/hummock_manager_tests.rs b/src/meta/src/hummock/hummock_manager_tests.rs index b8419172ab304..1f55d270600ca 100644 --- a/src/meta/src/hummock/hummock_manager_tests.rs +++ b/src/meta/src/hummock/hummock_manager_tests.rs @@ -18,15 +18,18 @@ use std::time::Duration; use itertools::Itertools; use risingwave_common::util::epoch::INVALID_EPOCH; use risingwave_hummock_sdk::compact::compact_task_to_string; +use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; +// use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::{ HummockContextId, HummockSSTableId, FIRST_VERSION_ID, INVALID_VERSION_ID, }; use risingwave_pb::common::{HostAddress, ParallelUnitType, WorkerType}; use risingwave_pb::hummock::{ - HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, HummockVersion, + HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, HummockVersion, KeyRange, }; +use crate::hummock::compaction::ManualCompactionOption; use crate::hummock::error::Error; use crate::hummock::model::CurrentHummockVersionId; use crate::hummock::test_utils::*; @@ -56,10 +59,12 @@ async fn test_hummock_pin_unpin() { .pin_version(context_id, u64::MAX) .await .unwrap(); + let levels = hummock_version + .get_compaction_group_levels(StaticCompactionGroupId::StateDefault.into()); assert_eq!(version_id, hummock_version.id); - assert_eq!(7, hummock_version.levels.len()); - assert_eq!(0, hummock_version.levels[0].table_infos.len()); - assert_eq!(0, hummock_version.levels[1].table_infos.len()); + assert_eq!(7, levels.len()); + assert_eq!(0, levels[0].table_infos.len()); + assert_eq!(0, levels[1].table_infos.len()); let pinned_versions = HummockPinnedVersion::list(env.meta_store()).await.unwrap(); assert_eq!(pin_versions_sum(&pinned_versions), 1); @@ -181,7 +186,7 @@ async fn test_hummock_compaction_task() { let epoch: u64 = 1; let original_tables = generate_test_tables(epoch, get_sst_ids(&hummock_manager, sst_num).await); hummock_manager - .commit_epoch(epoch, original_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&original_tables)) .await .unwrap(); @@ -219,8 +224,8 @@ async fn test_hummock_compaction_task() { assert_eq!(compact_task.get_task_id(), 2); // In the test case, we assume that each SST contains data of 2 relational tables, and // one of them overlaps with the previous SST. So there will be one more relational tables - // (for vnode mapping) than SSTs. - assert_eq!(compact_task.get_vnode_mappings().len(), sst_num + 1); + // (for vnode mapping) than SSTs. but we now remove vnode mapping in compact task + assert_eq!(compact_task.get_vnode_mappings().len(), 0); // Cancel the task and succeed. compact_task.task_status = false; @@ -295,7 +300,7 @@ async fn test_hummock_table() { let epoch: u64 = 1; let original_tables = generate_test_tables(epoch, get_sst_ids(&hummock_manager, 2).await); hummock_manager - .commit_epoch(epoch, original_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&original_tables)) .await .unwrap(); @@ -306,7 +311,7 @@ async fn test_hummock_table() { assert_eq!( Ordering::Equal, pinned_version - .levels + .get_compaction_group_levels(StaticCompactionGroupId::StateDefault.into()) .iter() .flat_map(|level| level.table_infos.iter()) .map(|info| info.id) @@ -349,7 +354,7 @@ async fn test_hummock_transaction() { // Commit epoch1 hummock_manager - .commit_epoch(epoch1, tables_in_epoch1.clone()) + .commit_epoch(epoch1, to_local_sstable_info(&tables_in_epoch1)) .await .unwrap(); committed_tables.extend(tables_in_epoch1.clone()); @@ -396,7 +401,7 @@ async fn test_hummock_transaction() { // Commit epoch2 hummock_manager - .commit_epoch(epoch2, tables_in_epoch2.clone()) + .commit_epoch(epoch2, to_local_sstable_info(&tables_in_epoch2)) .await .unwrap(); committed_tables.extend(tables_in_epoch2); @@ -559,7 +564,7 @@ async fn test_hummock_manager_basic() { let epoch: u64 = 1; let original_tables = generate_test_tables(epoch, get_sst_ids(&hummock_manager, 2).await); hummock_manager - .commit_epoch(epoch, original_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&original_tables)) .await .unwrap(); @@ -660,7 +665,7 @@ async fn test_retryable_pin_version() { ); // Increase the version hummock_manager - .commit_epoch(epoch, test_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&test_tables)) .await .unwrap(); epoch += 1; @@ -693,7 +698,7 @@ async fn test_retryable_pin_version() { ); // Increase the version hummock_manager - .commit_epoch(epoch, test_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&test_tables)) .await .unwrap(); epoch += 1; @@ -731,7 +736,7 @@ async fn test_pin_snapshot_response_lost() { ); // [ ] -> [ e0 ] hummock_manager - .commit_epoch(epoch, test_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&test_tables)) .await .unwrap(); epoch += 1; @@ -758,7 +763,7 @@ async fn test_pin_snapshot_response_lost() { ); // [ e0:pinned ] -> [ e0:pinned, e1 ] hummock_manager - .commit_epoch(epoch, test_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&test_tables)) .await .unwrap(); epoch += 1; @@ -794,7 +799,7 @@ async fn test_pin_snapshot_response_lost() { ); // [ e0, e1:pinned ] -> [ e0, e1:pinned, e2 ] hummock_manager - .commit_epoch(epoch, test_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&test_tables)) .await .unwrap(); epoch += 1; @@ -821,7 +826,7 @@ async fn test_pin_snapshot_response_lost() { ); // [ e0, e1:pinned, e2:pinned ] -> [ e0, e1:pinned, e2:pinned, e3 ] hummock_manager - .commit_epoch(epoch, test_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&test_tables)) .await .unwrap(); epoch += 1; @@ -847,7 +852,7 @@ async fn test_print_compact_task() { let epoch: u64 = 1; let original_tables = generate_test_tables(epoch, get_sst_ids(&hummock_manager, 2).await); hummock_manager - .commit_epoch(epoch, original_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&original_tables)) .await .unwrap(); @@ -876,7 +881,7 @@ async fn test_invalid_sst_id() { let epoch = 1; let ssts = generate_test_tables(epoch, vec![HummockSSTableId::MAX]); let error = hummock_manager - .commit_epoch(epoch, ssts.clone()) + .commit_epoch(epoch, to_local_sstable_info(&ssts)) .await .unwrap_err(); assert!(matches!(error, Error::InternalError(_))); @@ -910,7 +915,7 @@ async fn test_mark_orphan_ssts() { ); // Cannot commit_epoch for marked SST ids. let error = hummock_manager - .commit_epoch(epoch, ssts.clone()) + .commit_epoch(epoch, to_local_sstable_info(&ssts)) .await .unwrap_err(); assert!(matches!(error, Error::InternalError(_))); @@ -934,9 +939,10 @@ async fn test_trigger_manual_compaction() { } { + let option = ManualCompactionOption::default(); // to check no compactor let result = hummock_manager - .trigger_manual_compaction(StaticCompactionGroupId::StateDefault.into()) + .trigger_manual_compaction(StaticCompactionGroupId::StateDefault.into(), option) .await; assert_eq!( @@ -949,8 +955,9 @@ async fn test_trigger_manual_compaction() { let compactor_manager_ref = hummock_manager.compactor_manager_ref_for_test(); let receiver = compactor_manager_ref.add_compactor(context_id); { + let option = ManualCompactionOption::default(); let result = hummock_manager - .trigger_manual_compaction(StaticCompactionGroupId::StateDefault.into()) + .trigger_manual_compaction(StaticCompactionGroupId::StateDefault.into(), option) .await; assert_eq!("internal error: trigger_manual_compaction No compaction_task is available. compaction_group 2", result.err().unwrap().to_string()); } @@ -959,7 +966,7 @@ async fn test_trigger_manual_compaction() { let epoch: u64 = 1; let original_tables = generate_test_tables(epoch, get_sst_ids(&hummock_manager, sst_num).await); hummock_manager - .commit_epoch(epoch, original_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&original_tables)) .await .unwrap(); @@ -980,8 +987,9 @@ async fn test_trigger_manual_compaction() { // to check compactor send task fail drop(receiver); { + let option = ManualCompactionOption::default(); let result = hummock_manager - .trigger_manual_compaction(StaticCompactionGroupId::StateDefault.into()) + .trigger_manual_compaction(StaticCompactionGroupId::StateDefault.into(), option) .await; assert!(result.is_err()); } @@ -991,8 +999,17 @@ async fn test_trigger_manual_compaction() { let _receiver = compactor_manager_ref.add_compactor(context_id); { + let option = ManualCompactionOption { + level: 0, + key_range: KeyRange { + inf: true, + ..Default::default() + }, + ..Default::default() + }; + let result = hummock_manager - .trigger_manual_compaction(StaticCompactionGroupId::StateDefault.into()) + .trigger_manual_compaction(StaticCompactionGroupId::StateDefault.into(), option) .await; assert!(result.is_ok()); } @@ -1007,9 +1024,10 @@ async fn test_trigger_manual_compaction() { assert_eq!(task_id, compact_task.task_id); { + let option = ManualCompactionOption::default(); // all sst pending , test no compaction avail let result = hummock_manager - .trigger_manual_compaction(StaticCompactionGroupId::StateDefault.into()) + .trigger_manual_compaction(StaticCompactionGroupId::StateDefault.into(), option) .await; assert!(result.is_err()); } diff --git a/src/meta/src/hummock/metrics_utils.rs b/src/meta/src/hummock/metrics_utils.rs index 67e853b835a98..7834ecffce3e7 100644 --- a/src/meta/src/hummock/metrics_utils.rs +++ b/src/meta/src/hummock/metrics_utils.rs @@ -17,6 +17,8 @@ use std::time::{SystemTime, UNIX_EPOCH}; use itertools::enumerate; use prost::Message; +use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; +use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_pb::hummock::HummockVersion; use crate::hummock::compaction::CompactStatus; @@ -36,9 +38,11 @@ pub fn trigger_sst_stat( compact_status: &CompactStatus, current_version: &HummockVersion, ) { - let level_sst_cnt = |level_idx: usize| current_version.levels[level_idx].table_infos.len(); - let level_sst_size = - |level_idx: usize| current_version.levels[level_idx].total_file_size / 1024; + // TODO #2065: add metrics for all compaction groups + let levels = + current_version.get_compaction_group_levels(StaticCompactionGroupId::StateDefault.into()); + let level_sst_cnt = |level_idx: usize| levels[level_idx].table_infos.len(); + let level_sst_size = |level_idx: usize| levels[level_idx].total_file_size / 1024; for (idx, level_handler) in enumerate(compact_status.level_handlers.iter()) { let sst_num = level_sst_cnt(idx); let compact_cnt = level_handler.get_pending_file_count(); diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index 3b59d010010b2..ad5b5f4948771 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -17,10 +17,12 @@ use std::sync::Arc; use anyhow::anyhow; use async_trait::async_trait; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; -use risingwave_hummock_sdk::{HummockContextId, HummockEpoch, HummockSSTableId, HummockVersionId}; +use risingwave_hummock_sdk::{ + HummockContextId, HummockEpoch, HummockSSTableId, HummockVersionId, LocalSstableInfo, +}; use risingwave_pb::hummock::{ - CompactTask, CompactionGroup, HummockSnapshot, HummockVersion, SstableInfo, - SubscribeCompactTasksResponse, VacuumTask, + CompactTask, CompactionGroup, HummockSnapshot, HummockVersion, SubscribeCompactTasksResponse, + VacuumTask, }; use risingwave_rpc_client::error::{Result, RpcError}; use risingwave_rpc_client::HummockMetaClient; @@ -121,7 +123,11 @@ impl HummockMetaClient for MockHummockMetaClient { .map_err(mock_err) } - async fn commit_epoch(&self, epoch: HummockEpoch, sstables: Vec) -> Result<()> { + async fn commit_epoch( + &self, + epoch: HummockEpoch, + sstables: Vec, + ) -> Result<()> { self.hummock_manager .commit_epoch(epoch, sstables) .await @@ -140,7 +146,12 @@ impl HummockMetaClient for MockHummockMetaClient { todo!() } - async fn trigger_manual_compaction(&self, _compaction_group_id: u64) -> Result<()> { + async fn trigger_manual_compaction( + &self, + _compaction_group_id: u64, + _table_id: u32, + _level: u32, + ) -> Result<()> { todo!() } } diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 6eda633825d4f..8ea567fa2d194 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -16,10 +16,11 @@ use std::sync::Arc; use std::time::Duration; use itertools::Itertools; +use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::key_with_epoch; -use risingwave_hummock_sdk::{HummockContextId, HummockEpoch, HummockSSTableId}; -use risingwave_pb::common::{HostAddress, VNodeBitmap, WorkerNode, WorkerType}; +use risingwave_hummock_sdk::{HummockContextId, HummockEpoch, HummockSSTableId, LocalSstableInfo}; +use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; use risingwave_pb::hummock::{HummockVersion, KeyRange, SstableInfo}; use crate::cluster::{ClusterManager, ClusterManagerRef}; @@ -30,6 +31,12 @@ use crate::manager::MetaSrvEnv; use crate::rpc::metrics::MetaMetrics; use crate::storage::{MemStore, MetaStore}; +pub fn to_local_sstable_info(ssts: &[SstableInfo]) -> Vec { + ssts.iter() + .map(|sst| (StaticCompactionGroupId::StateDefault.into(), sst.clone())) + .collect_vec() +} + pub async fn add_test_tables( hummock_manager: &HummockManager, context_id: HummockContextId, @@ -46,7 +53,7 @@ where ]; let test_tables = generate_test_tables(epoch, table_ids); hummock_manager - .commit_epoch(epoch, test_tables.clone()) + .commit_epoch(epoch, to_local_sstable_info(&test_tables)) .await .unwrap(); // Current state: {v0: [], v1: [test_tables]} @@ -80,7 +87,7 @@ where vec![hummock_manager.get_new_table_id().await.unwrap()], ); hummock_manager - .commit_epoch(epoch, test_tables_3.clone()) + .commit_epoch(epoch, to_local_sstable_info(&test_tables_3)) .await .unwrap(); // Current state: {v0: [], v1: [test_tables], v2: [test_tables_2, to_delete:test_tables], v3: @@ -99,16 +106,7 @@ pub fn generate_test_tables(epoch: u64, sst_ids: Vec) -> Vec Vec pub fn get_sorted_committed_sstable_ids(hummock_version: &HummockVersion) -> Vec { hummock_version - .levels + .get_compaction_group_levels(StaticCompactionGroupId::StateDefault.into()) .iter() .flat_map(|level| level.table_infos.iter().map(|info| info.id)) .sorted() diff --git a/src/meta/src/manager/hash_mapping.rs b/src/meta/src/manager/hash_mapping.rs index 898fc7f2c5e34..9705959c373f2 100644 --- a/src/meta/src/manager/hash_mapping.rs +++ b/src/meta/src/manager/hash_mapping.rs @@ -16,11 +16,10 @@ use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; use parking_lot::Mutex; -use risingwave_common::hash::{VirtualNode, VIRTUAL_NODE_COUNT}; +use risingwave_common::types::{ParallelUnitId, VirtualNode, VIRTUAL_NODE_COUNT}; use risingwave_pb::common::ParallelUnit; use super::TableId; -use crate::cluster::ParallelUnitId; use crate::model::FragmentId; pub type HashMappingManagerRef = Arc; @@ -109,7 +108,7 @@ impl HashMappingManager { /// `HashMappingInfo` stores the vnode mapping and some other helpers for maintaining a /// load-balanced vnode mapping. -#[derive(Clone)] +#[derive(Clone, Debug)] struct HashMappingInfo { /// Hash mapping from virtual node to parallel unit. vnode_mapping: Vec, @@ -180,6 +179,7 @@ impl HashMappingManagerCore { owner_mapping, load_balancer, }; + self.hash_mapping_infos.insert(fragment_id, mapping_info); vnode_mapping @@ -222,7 +222,7 @@ impl HashMappingManagerCore { #[cfg(test)] mod tests { use itertools::Itertools; - use risingwave_common::hash::VIRTUAL_NODE_COUNT; + use risingwave_common::types::VIRTUAL_NODE_COUNT; use risingwave_pb::common::{ParallelUnit, ParallelUnitType}; use super::{HashMappingInfo, HashMappingManager}; diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 1f1dca0874edd..aefbb4c34cc99 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -17,6 +17,7 @@ use std::collections::{BTreeMap, HashMap, HashSet, VecDeque}; use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::error::Result; +use risingwave_common::types::ParallelUnitId; use risingwave_pb::meta::table_fragments::{ActorState, ActorStatus, Fragment}; use risingwave_pb::meta::TableFragments as ProstTableFragments; use risingwave_pb::stream_plan::source_node::SourceType; @@ -24,7 +25,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{FragmentType, StreamActor, StreamNode}; use super::{ActorId, FragmentId}; -use crate::cluster::{ParallelUnitId, WorkerId}; +use crate::cluster::WorkerId; use crate::manager::SourceId; use crate::model::MetadataModel; diff --git a/src/meta/src/rpc/server.rs b/src/meta/src/rpc/server.rs index 0c1d998e8a626..0a844b69a05c0 100644 --- a/src/meta/src/rpc/server.rs +++ b/src/meta/src/rpc/server.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use std::time::{Duration, SystemTime, UNIX_EPOCH}; use etcd_client::{Client as EtcdClient, ConnectOptions}; +use itertools::Itertools; use prost::Message; use risingwave_common::error::ErrorCode::InternalError; use risingwave_common::error::{Result, RwError}; @@ -336,6 +337,7 @@ pub async fn rpc_serve_with_store( barrier_manager.clone(), catalog_manager.clone(), fragment_manager.clone(), + compaction_group_manager.clone(), ) .await .unwrap(), @@ -360,6 +362,24 @@ pub async fn rpc_serve_with_store( .unwrap(), ); + compaction_group_manager + .purge_stale_members( + &fragment_manager + .list_table_fragments() + .await + .expect("list_table_fragments"), + &catalog_manager + .get_catalog_core_guard() + .await + .list_sources() + .await + .expect("list_sources") + .into_iter() + .map(|source| source.id) + .collect_vec(), + ) + .await + .unwrap(); let compaction_scheduler = Arc::new(CompactionScheduler::new( hummock_manager.clone(), compactor_manager.clone(), @@ -386,6 +406,7 @@ pub async fn rpc_serve_with_store( compactor_manager.clone(), vacuum_trigger.clone(), compaction_group_manager.clone(), + fragment_manager.clone(), ); let notification_manager = env.notification_manager_ref(); let notification_srv = diff --git a/src/meta/src/rpc/service/ddl_service.rs b/src/meta/src/rpc/service/ddl_service.rs index 99b251571f75d..605ccf230f87c 100644 --- a/src/meta/src/rpc/service/ddl_service.rs +++ b/src/meta/src/rpc/service/ddl_service.rs @@ -12,13 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashSet; +use std::collections::{HashMap, HashSet}; use risingwave_common::catalog::CatalogVersion; -use risingwave_common::error::{tonic_err, Result as RwResult}; +use risingwave_common::error::{tonic_err, ErrorCode, Result as RwResult}; +use risingwave_common::util::compress::compress_data; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::*; -use risingwave_pb::common::ParallelUnitType; +use risingwave_pb::common::{ParallelUnitMapping, ParallelUnitType}; use risingwave_pb::ddl_service::ddl_service_server::DdlService; use risingwave_pb::ddl_service::*; use risingwave_pb::plan_common::TableRefId; @@ -28,7 +29,7 @@ use tonic::{Request, Response, Status}; use crate::cluster::ClusterManagerRef; use crate::manager::{CatalogManagerRef, IdCategory, MetaSrvEnv, SourceId, TableId}; -use crate::model::TableFragments; +use crate::model::{FragmentId, TableFragments}; use crate::storage::MetaStore; use crate::stream::{ ActorGraphBuilder, FragmentManagerRef, GlobalStreamManagerRef, SourceManagerRef, @@ -294,6 +295,8 @@ where .await .map_err(tonic_err)?; return Err(e.into()); + } else { + self.set_mview_mapping(&mut mview).map_err(tonic_err)?; } // 4. Finally, update the catalog. @@ -376,6 +379,17 @@ where version, })) } + + async fn list_materialized_view( + &self, + _request: Request, + ) -> Result, Status> { + use crate::model::MetadataModel; + let tables = Table::list(self.env.meta_store()) + .await + .map_err(tonic_err)?; + Ok(Response::new(ListMaterializedViewResponse { tables })) + } } impl DdlServiceImpl @@ -426,14 +440,32 @@ where affiliated_source, ..Default::default() }; - let graph = ActorGraphBuilder::generate_graph( - self.env.id_gen_manager_ref(), - self.fragment_manager.clone(), - parallel_degree as u32, - &fragment_graph, - &mut ctx, - ) - .await?; + + let mut actor_graph_builder = + ActorGraphBuilder::new(self.env.id_gen_manager_ref(), &fragment_graph, &mut ctx) + .await?; + + // TODO(Kexiang): now simply use Count(ParallelUnit) - 1 as parallelism of each fragment + let parallelisms: HashMap = actor_graph_builder + .list_fragment_ids() + .into_iter() + .map(|(fragment_id, is_singleton)| { + if is_singleton { + (fragment_id, 1) + } else { + (fragment_id, parallel_degree as u32) + } + }) + .collect(); + + let graph = actor_graph_builder + .generate_graph( + self.env.id_gen_manager_ref(), + self.fragment_manager.clone(), + parallelisms, + &mut ctx, + ) + .await?; assert_eq!( fragment_graph.table_ids_cnt, ctx.internal_table_id_set.len() as u32 @@ -524,6 +556,8 @@ where // drop previously created source self.source_manager.drop_source(source_id).await?; return Err(e); + } else { + self.set_mview_mapping(&mut mview).map_err(tonic_err)?; } // Finally, update the catalog. @@ -557,4 +591,27 @@ where Ok(version) } + + /// Fill in mview's vnode mapping so that frontend will know the data distribution. + fn set_mview_mapping(&self, mview: &mut Table) -> RwResult<()> { + let vnode_mapping = self + .env + .hash_mapping_manager_ref() + .get_table_hash_mapping(&mview.id); + match vnode_mapping { + Some(vnode_mapping) => { + let (original_indices, data) = compress_data(&vnode_mapping); + mview.mapping = Some(ParallelUnitMapping { + table_id: mview.id, + original_indices, + data, + }); + Ok(()) + } + None => Err(ErrorCode::InternalError( + "no data distribution found for materialized view".to_string(), + ) + .into()), + } + } } diff --git a/src/meta/src/rpc/service/hummock_service.rs b/src/meta/src/rpc/service/hummock_service.rs index b682307bd366f..97bdad6b0e014 100644 --- a/src/meta/src/rpc/service/hummock_service.rs +++ b/src/meta/src/rpc/service/hummock_service.rs @@ -12,17 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; use std::sync::Arc; +use risingwave_common::catalog::TableId; use risingwave_common::error::{tonic_err, ErrorCode}; use risingwave_pb::hummock::hummock_manager_service_server::HummockManagerService; use risingwave_pb::hummock::*; use tonic::{Request, Response, Status}; +use crate::hummock::compaction::ManualCompactionOption; use crate::hummock::compaction_group::manager::CompactionGroupManagerRef; use crate::hummock::{CompactorManagerRef, HummockManagerRef, VacuumTrigger}; use crate::rpc::service::RwReceiverStream; use crate::storage::MetaStore; +use crate::stream::FragmentManagerRef; pub struct HummockServiceImpl where @@ -32,6 +36,7 @@ where compactor_manager: CompactorManagerRef, vacuum_trigger: Arc>, compaction_group_manager: CompactionGroupManagerRef, + fragment_manager: FragmentManagerRef, } impl HummockServiceImpl @@ -43,12 +48,14 @@ where compactor_manager: CompactorManagerRef, vacuum_trigger: Arc>, compaction_group_manager: CompactionGroupManagerRef, + fragment_manager: FragmentManagerRef, ) -> Self { HummockServiceImpl { hummock_manager, compactor_manager, vacuum_trigger, compaction_group_manager, + fragment_manager, } } } @@ -230,10 +237,47 @@ where &self, request: Request, ) -> Result, Status> { - let compaction_group_id = request.into_inner().compaction_group_id; + let request = request.into_inner(); + let compaction_group_id = request.compaction_group_id; + let mut option = ManualCompactionOption { + level: request.level as usize, + ..Default::default() + }; + + // rewrite the key_range + match request.key_range { + Some(key_range) => { + option.key_range = key_range; + } + + None => { + option.key_range = KeyRange { + inf: true, + ..Default::default() + } + } + } + + // get internal_table_id by fragment_manager + let table_id = TableId::new(request.table_id); + if let Ok(table_frgament) = self + .fragment_manager + .select_table_fragments_by_table_id(&table_id) + .await + { + option.internal_table_id = HashSet::from_iter(table_frgament.internal_table_ids()); + } + option.internal_table_id.insert(request.table_id); // need to handle outter table_id (mv) + + tracing::info!( + "Try trigger_manual_compaction compaction_group_id {} option {:?}", + compaction_group_id, + &option + ); + let result_state = match self .hummock_manager - .trigger_manual_compaction(compaction_group_id) + .trigger_manual_compaction(compaction_group_id, option) .await { Ok(_) => None, diff --git a/src/meta/src/stream/meta.rs b/src/meta/src/stream/meta.rs index 0eee0c01b1334..2d7bf22463942 100644 --- a/src/meta/src/stream/meta.rs +++ b/src/meta/src/stream/meta.rs @@ -16,18 +16,17 @@ use std::collections::hash_map::Entry; use std::collections::{BTreeMap, HashMap, HashSet}; use std::sync::Arc; -use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::error::ErrorCode::InternalError; use risingwave_common::error::{Result, RwError}; -use risingwave_common::hash::VIRTUAL_NODE_COUNT; use risingwave_common::try_match_expand; +use risingwave_common::types::{ParallelUnitId, VIRTUAL_NODE_COUNT}; use risingwave_common::util::compress::decompress_data; use risingwave_pb::meta::table_fragments::ActorState; use risingwave_pb::stream_plan::{FragmentType, StreamActor}; use tokio::sync::RwLock; -use crate::cluster::{ParallelUnitId, WorkerId}; +use crate::cluster::WorkerId; use crate::hummock::compaction_group::manager::CompactionGroupManagerRef; use crate::manager::{HashMappingManagerRef, MetaSrvEnv}; use crate::model::{ActorId, MetadataModel, TableFragments, Transactional}; @@ -85,11 +84,6 @@ where Self::restore_vnode_mappings(env.hash_mapping_manager_ref(), &table_fragments)?; - let table_fragments_list = table_fragments.values().collect_vec(); - compaction_group_manager - .purge_stale_members(&table_fragments_list) - .await?; - Ok(Self { meta_store, core: RwLock::new(FragmentManagerCore { table_fragments }), @@ -147,8 +141,7 @@ where )))), Entry::Vacant(v) => { // Register to compaction group beforehand. - // If any following operation fails, the registration will be eventually reverted by - // CompactionGroupManager::purge_stale_members. + // If any following operation fails, the registration will be eventually reverted. self.compaction_group_manager .register_table_fragments(&table_fragment) .await?; @@ -174,7 +167,11 @@ where .unregister_table_fragments(&table_fragments) .await { - tracing::warn!("Failed to unregister table {}. It wll be unregistered eventually by CompactionGroupManager::purge_stale_members.\n{:#?}", table_id, e); + tracing::warn!( + "Failed to unregister table {}. It wll be unregistered eventually.\n{:#?}", + table_id, + e + ); } Ok(()) } @@ -288,7 +285,11 @@ where .unregister_table_fragments(&table_fragments) .await { - tracing::warn!("Failed to unregister table {}. It wll be unregistered eventually by CompactionGroupManager::purge_stale_members.\n{:#?}", table_id, e); + tracing::warn!( + "Failed to unregister table {}. It wll be unregistered eventually.\n{:#?}", + table_id, + e + ); } Ok(()) } else { diff --git a/src/meta/src/stream/scheduler.rs b/src/meta/src/stream/scheduler.rs index ccb19b50adbd6..8246e9fd50743 100644 --- a/src/meta/src/stream/scheduler.rs +++ b/src/meta/src/stream/scheduler.rs @@ -15,9 +15,10 @@ use std::collections::{BTreeMap, HashMap}; use std::sync::atomic::{AtomicUsize, Ordering}; +use risingwave_common::buffer::BitmapBuilder; use risingwave_common::error::ErrorCode::InternalError; use risingwave_common::error::{internal_error, Result}; -use risingwave_common::hash::VNODE_BITMAP_LEN; +use risingwave_common::types::VIRTUAL_NODE_COUNT; use risingwave_common::util::compress::compress_data; use risingwave_pb::common::{ActorInfo, ParallelUnit, ParallelUnitMapping, ParallelUnitType}; use risingwave_pb::meta::table_fragments::fragment::FragmentDistributionType; @@ -202,10 +203,13 @@ where // Normal fragment // Find out all the hash parallel units in the cluster. - let parallel_units = self + let mut parallel_units = self .cluster_manager .list_parallel_units(Some(ParallelUnitType::Hash)) .await; + // FIXME(Kexiang): select appropriate parallel_units, currently only support + // `parallel_degree < parallel_units.size()` + parallel_units.truncate(fragment.actors.len()); // Build vnode mapping according to the parallel units. self.set_fragment_vnode_mapping(fragment, ¶llel_units)?; @@ -215,6 +219,7 @@ where .hash_mapping_manager .get_fragment_hash_mapping(&fragment.fragment_id) .unwrap(); + let mut vnode_bitmaps = HashMap::new(); vnode_mapping .iter() @@ -222,24 +227,31 @@ where .for_each(|(vnode, parallel_unit)| { vnode_bitmaps .entry(*parallel_unit) - .or_insert([0; VNODE_BITMAP_LEN])[(vnode >> 3) as usize] |= - 1 << (vnode & 0b111); + .or_insert_with(|| BitmapBuilder::zeroed(VIRTUAL_NODE_COUNT)) + .set(vnode, true); }); + let vnode_bitmaps = vnode_bitmaps + .into_iter() + .map(|(u, b)| (u, b.finish())) + .collect::>(); // Record actor locations and set vnodes into the actors. for (idx, actor) in fragment.actors.iter_mut().enumerate() { if actor.same_worker_node_as_upstream && !actor.upstream_actor_id.is_empty() { let parallel_unit = locations.schedule_colocate_with(&actor.upstream_actor_id)?; - actor.vnode_bitmap = vnode_bitmaps.get(¶llel_unit.id).unwrap().to_vec(); + actor.vnode_bitmap = + Some(vnode_bitmaps.get(¶llel_unit.id).unwrap().to_protobuf()); locations .actor_locations .insert(actor.actor_id, parallel_unit); } else { - actor.vnode_bitmap = vnode_bitmaps - .get(¶llel_units[idx % parallel_units.len()].id) - .unwrap() - .to_vec(); + actor.vnode_bitmap = Some( + vnode_bitmaps + .get(¶llel_units[idx % parallel_units.len()].id) + .unwrap() + .to_protobuf(), + ); locations.actor_locations.insert( actor.actor_id, parallel_units[idx % parallel_units.len()].clone(), @@ -288,7 +300,8 @@ mod test { use std::time::Duration; use itertools::Itertools; - use risingwave_common::hash::VIRTUAL_NODE_COUNT; + use risingwave_common::buffer::Bitmap; + use risingwave_common::types::VIRTUAL_NODE_COUNT; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::meta::table_fragments::fragment::FragmentDistributionType; use risingwave_pb::plan_common::TableRefId; @@ -339,7 +352,7 @@ mod test { dispatcher: vec![], upstream_actor_id: vec![], same_worker_node_as_upstream: false, - vnode_bitmap: vec![], + vnode_bitmap: None, }], vnode_mapping: None, }; @@ -368,7 +381,7 @@ mod test { dispatcher: vec![], upstream_actor_id: vec![], same_worker_node_as_upstream: false, - vnode_bitmap: vec![], + vnode_bitmap: None, }) .collect_vec(); actor_id += node_count * 7; @@ -404,7 +417,7 @@ mod test { None ); for actor in fragment.actors { - assert!(actor.vnode_bitmap.is_empty()); + assert!(actor.vnode_bitmap.is_none()); } } @@ -440,10 +453,7 @@ mod test { ); let mut vnode_sum = 0; for actor in fragment.actors { - assert!(!actor.vnode_bitmap.is_empty()); - for byte in actor.vnode_bitmap { - vnode_sum += byte.count_ones(); - } + vnode_sum += Bitmap::try_from(actor.get_vnode_bitmap()?)?.num_high_bits(); } assert_eq!(vnode_sum as usize, VIRTUAL_NODE_COUNT); } diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 35c1f4066277c..48d9a1a4e41ed 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -48,6 +48,7 @@ use tokio_retry::strategy::FixedInterval; use crate::barrier::{BarrierManagerRef, Command}; use crate::cluster::ClusterManagerRef; +use crate::hummock::compaction_group::manager::CompactionGroupManagerRef; use crate::manager::{CatalogManagerRef, MetaSrvEnv, SourceId}; use crate::model::{ActorId, FragmentId, MetadataModel, TableFragments, Transactional}; use crate::storage::{MetaStore, Transaction}; @@ -63,6 +64,7 @@ pub struct SourceManager { cluster_manager: ClusterManagerRef, catalog_manager: CatalogManagerRef, barrier_manager: BarrierManagerRef, + compaction_group_manager: CompactionGroupManagerRef, core: Arc>>, } @@ -408,6 +410,7 @@ where barrier_manager: BarrierManagerRef, catalog_manager: CatalogManagerRef, fragment_manager: FragmentManagerRef, + compaction_group_manager: CompactionGroupManagerRef, ) -> Result { let mut managed_sources = HashMap::new(); { @@ -444,6 +447,7 @@ where cluster_manager, catalog_manager, barrier_manager, + compaction_group_manager, core, }) } @@ -573,6 +577,10 @@ where /// Broadcast the create source request to all compute nodes. pub async fn create_source(&self, source: &Source) -> Result<()> { + // Register beforehand and is safeguarded by CompactionGroupManager::purge_stale_members. + self.compaction_group_manager + .register_source(source.id) + .await?; let futures = self .all_stream_clients() .await? @@ -646,6 +654,19 @@ where ); } + // Unregister afterwards and is safeguarded by CompactionGroupManager::purge_stale_members. + if let Err(e) = self + .compaction_group_manager + .unregister_source(source_id) + .await + { + tracing::warn!( + "Failed to unregister source {}. It wll be unregistered eventually.\n{:#?}", + source_id, + e + ); + } + Ok(()) } diff --git a/src/meta/src/stream/stream_graph.rs b/src/meta/src/stream/stream_graph.rs index 4f816571a0917..c2923d7466a59 100644 --- a/src/meta/src/stream/stream_graph.rs +++ b/src/meta/src/stream/stream_graph.rs @@ -336,7 +336,7 @@ impl StreamActorBuilder { }, )| *same_worker_node, ), - vnode_bitmap: vec![], + vnode_bitmap: None, } } } @@ -688,61 +688,80 @@ impl BuildActorGraphState { /// [`ActorGraphBuilder`] generates the proto for interconnected actors for a streaming pipeline. pub struct ActorGraphBuilder { - /// degree of parallelism - parallel_degree: u32, + /// GlobalFragmentId -> parallel_degree + parallelisms: Option>, + + fragment_graph: StreamFragmentGraph, } impl ActorGraphBuilder { + pub async fn new( + id_gen_manager: IdGeneratorManagerRef, + fragment_graph: &StreamFragmentGraphProto, + ctx: &mut CreateMaterializedViewContext, + ) -> Result + where + S: MetaStore, + { + // save dependent table ids in ctx + ctx.dependent_table_ids = fragment_graph + .dependent_table_ids + .iter() + .map(|table_id| TableId::new(*table_id)) + .collect(); + + let fragment_len = fragment_graph.fragments.len() as u32; + let offset = id_gen_manager + .generate_interval::<{ IdCategory::Fragment }>(fragment_len as i32) + .await? as _; + + // Compute how many table ids should be allocated for all actors. + // Allocate all needed table ids for current MV. + let table_ids_cnt = fragment_graph.table_ids_cnt; + let start_table_id = id_gen_manager + .generate_interval::<{ IdCategory::Table }>(table_ids_cnt as i32) + .await? as _; + ctx.table_id_offset = start_table_id; + + Ok(Self { + fragment_graph: StreamFragmentGraph::from_protobuf(fragment_graph.clone(), offset), + parallelisms: None, + }) + } + pub async fn generate_graph( + &mut self, id_gen_manager: IdGeneratorManagerRef, fragment_manager: FragmentManagerRef, - parallel_degree: u32, - fragment_graph: &StreamFragmentGraphProto, + parallelisms: HashMap, ctx: &mut CreateMaterializedViewContext, ) -> Result> where S: MetaStore, { - Self { parallel_degree } - .generate_graph_inner(id_gen_manager, fragment_manager, fragment_graph, ctx) + self.parallelisms = Some(parallelisms); + self.generate_graph_inner(id_gen_manager, fragment_manager, ctx) .await } + pub fn list_fragment_ids(&self) -> Vec<(FragmentId, bool)> { + self.fragment_graph + .fragments() + .iter() + .map(|(id, fragment)| (id.as_global_id(), fragment.is_singleton)) + .collect_vec() + } + /// Build a stream graph by duplicating each fragment as parallel actors. async fn generate_graph_inner( - self, + &self, id_gen_manager: IdGeneratorManagerRef, fragment_manager: FragmentManagerRef, - fragment_graph: &StreamFragmentGraphProto, ctx: &mut CreateMaterializedViewContext, ) -> Result> where S: MetaStore, { - let fragment_graph = { - // save dependent table ids in ctx - ctx.dependent_table_ids = fragment_graph - .dependent_table_ids - .iter() - .map(|table_id| TableId::new(*table_id)) - .collect(); - - let fragment_len = fragment_graph.fragments.len() as u32; - let offset = id_gen_manager - .generate_interval::<{ IdCategory::Fragment }>(fragment_len as i32) - .await? as _; - - // Compute how many table ids should be allocated for all actors. - // Allocate all needed table ids for current MV. - let table_ids_cnt = fragment_graph.table_ids_cnt; - let start_table_id = id_gen_manager - .generate_interval::<{ IdCategory::Table }>(table_ids_cnt as i32) - .await? as _; - ctx.table_id_offset = start_table_id; - - StreamFragmentGraph::from_protobuf(fragment_graph.clone(), offset) - }; - let stream_graph = { let BuildActorGraphState { stream_graph_builder, @@ -760,7 +779,7 @@ impl ActorGraphBuilder { state.stream_graph_builder.fill_info(info); // Generate actors of the streaming plan - self.build_actor_graph(&mut state, &fragment_graph)?; + self.build_actor_graph(&mut state, &self.fragment_graph)?; state }; @@ -782,7 +801,7 @@ impl ActorGraphBuilder { let stream_graph = stream_graph .into_iter() .map(|(fragment_id, actors)| { - let fragment = fragment_graph.get_fragment(fragment_id).unwrap(); + let fragment = self.fragment_graph.get_fragment(fragment_id).unwrap(); let fragment_id = fragment_id.as_global_id(); ( fragment_id, @@ -859,11 +878,13 @@ impl ActorGraphBuilder { ) -> Result<()> { let current_fragment = fragment_graph.get_fragment(fragment_id).unwrap().clone(); - let parallel_degree = if current_fragment.is_singleton { - 1 - } else { - self.parallel_degree - }; + let parallel_degree = self + .parallelisms + .as_ref() + .unwrap() + .get(&fragment_id.as_global_id()) + .unwrap() + .to_owned(); let node = Arc::new(current_fragment.node.unwrap()); let actor_ids = state diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 73b6c112f5bfe..d600c4d8fcb2b 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -20,7 +20,7 @@ use itertools::Itertools; use log::{debug, info}; use risingwave_common::catalog::TableId; use risingwave_common::error::{internal_error, Result}; -use risingwave_common::hash::VIRTUAL_NODE_COUNT; +use risingwave_common::types::{ParallelUnitId, VIRTUAL_NODE_COUNT}; use risingwave_pb::catalog::Source; use risingwave_pb::common::{ActorInfo, ParallelUnitMapping, WorkerType}; use risingwave_pb::meta::table_fragments::{ActorState, ActorStatus}; @@ -34,7 +34,7 @@ use uuid::Uuid; use super::ScheduledLocations; use crate::barrier::{BarrierManagerRef, Command}; -use crate::cluster::{ClusterManagerRef, ParallelUnitId, WorkerId}; +use crate::cluster::{ClusterManagerRef, WorkerId}; use crate::manager::{HashMappingManagerRef, MetaSrvEnv}; use crate::model::{ActorId, DispatcherId, TableFragments}; use crate::storage::MetaStore; @@ -851,6 +851,9 @@ mod tests { meta_metrics.clone(), )); + let compaction_group_manager = + Arc::new(CompactionGroupManager::new(env.clone()).await?); + let source_manager = Arc::new( SourceManager::new( env.clone(), @@ -858,6 +861,7 @@ mod tests { barrier_manager.clone(), catalog_manager.clone(), fragment_manager.clone(), + compaction_group_manager.clone(), ) .await?, ); @@ -952,8 +956,7 @@ mod tests { .cloned() .unwrap() .clone(); - assert!(!scheduled_actor.vnode_bitmap.is_empty()); - scheduled_actor.vnode_bitmap.clear(); + scheduled_actor.vnode_bitmap.take().unwrap(); assert_eq!(scheduled_actor, actor); assert!(services .state @@ -1053,8 +1056,7 @@ mod tests { .get(&actor.get_actor_id()) .cloned() .unwrap(); - assert!(!scheduled_actor.vnode_bitmap.is_empty()); - scheduled_actor.vnode_bitmap.clear(); + scheduled_actor.vnode_bitmap.take().unwrap(); assert_eq!(scheduled_actor, actor); assert!(services .state diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index d00da29617cc7..bf1ec06bf0ffd 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -260,6 +260,8 @@ fn make_stream_node() -> StreamNode { async fn test_fragmenter() -> Result<()> { use risingwave_frontend::stream_fragmenter::StreamFragmenter; + use crate::model::FragmentId; + let env = MetaSrvEnv::for_test().await; let stream_node = make_stream_node(); let compaction_group_manager = Arc::new(CompactionGroupManager::new(env.clone()).await?); @@ -268,14 +270,31 @@ async fn test_fragmenter() -> Result<()> { let parallel_degree = 4; let mut ctx = CreateMaterializedViewContext::default(); let graph = StreamFragmenter::build_graph(stream_node); - let graph = ActorGraphBuilder::generate_graph( - env.id_gen_manager_ref(), - fragment_manager, - parallel_degree, - &graph, - &mut ctx, - ) - .await?; + + let mut actor_graph_builder = + ActorGraphBuilder::new(env.id_gen_manager_ref(), &graph, &mut ctx).await?; + + let parallelisms: HashMap = actor_graph_builder + .list_fragment_ids() + .into_iter() + .map(|(fragment_id, is_singleton)| { + if is_singleton { + (fragment_id, 1) + } else { + (fragment_id, parallel_degree as u32) + } + }) + .collect(); + + let graph = actor_graph_builder + .generate_graph( + env.id_gen_manager_ref(), + fragment_manager, + parallelisms.clone(), + &mut ctx, + ) + .await?; + let table_fragments = TableFragments::new(TableId::default(), graph, ctx.internal_table_id_set.clone()); let actors = table_fragments.actors(); diff --git a/src/risedevtool/src/bin/risedev-playground.rs b/src/risedevtool/src/bin/risedev-playground.rs index bc160ce44e820..aded0b994ff0b 100644 --- a/src/risedevtool/src/bin/risedev-playground.rs +++ b/src/risedevtool/src/bin/risedev-playground.rs @@ -29,10 +29,10 @@ use console::style; use indicatif::{MultiProgress, ProgressBar}; use risedev::util::{complete_spin, fail_spin}; use risedev::{ - preflight_check, AwsS3Config, CompactorService, ComputeNodeService, ConfigExpander, - ConfigureTmuxTask, EnsureStopService, ExecuteContext, FrontendService, GrafanaService, - JaegerService, KafkaService, MetaNodeService, MinioService, PrometheusService, ServiceConfig, - Task, ZooKeeperService, RISEDEV_SESSION_NAME, + compute_risectl_env, preflight_check, AwsS3Config, CompactorService, ComputeNodeService, + ConfigExpander, ConfigureTmuxTask, EnsureStopService, ExecuteContext, FrontendService, + GrafanaService, JaegerService, KafkaService, MetaNodeService, MinioService, PrometheusService, + ServiceConfig, Task, ZooKeeperService, RISEDEV_SESSION_NAME, }; use tempfile::tempdir; use yaml_rust::YamlEmitter; @@ -385,6 +385,16 @@ fn main() -> Result<()> { println!("-------------------------------"); println!(); + let risectl_env = match compute_risectl_env(&services) { + Ok(x) => x, + Err(_) => "".into(), + }; + + std::fs::write( + Path::new(&env::var("PREFIX_CONFIG")?).join("risectl-env"), + &risectl_env, + )?; + println!("All services started successfully."); print!("{}", log_buffer); diff --git a/src/risedevtool/src/lib.rs b/src/risedevtool/src/lib.rs index 5925cab1979cb..240ae05d86fd6 100644 --- a/src/risedevtool/src/lib.rs +++ b/src/risedevtool/src/lib.rs @@ -30,6 +30,8 @@ mod compose; pub use compose::*; mod compose_deploy; pub use compose_deploy::*; +mod risectl_env; +pub use risectl_env::*; mod task; pub mod util; diff --git a/src/risedevtool/src/risectl_env.rs b/src/risedevtool/src/risectl_env.rs new file mode 100644 index 0000000000000..4f2da1ec8c7c4 --- /dev/null +++ b/src/risedevtool/src/risectl_env.rs @@ -0,0 +1,44 @@ +// Copyright 2022 Singularity Data +// +// 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::process::Command; + +use anyhow::Result; + +use crate::{add_storage_backend, ServiceConfig}; + +pub fn compute_risectl_env(services: &HashMap) -> Result { + // Pick one of the compute node and generate risectl config + for item in services.values() { + if let ServiceConfig::ComputeNode(c) = item { + let mut cmd = Command::new("compute-node"); + add_storage_backend( + "risectl", + c.provide_minio.as_ref().unwrap(), + c.provide_aws_s3.as_ref().unwrap(), + false, + &mut cmd, + )?; + let meta_node = &c.provide_meta_node.as_ref().unwrap()[0]; + return Ok(format!( + "export RW_HUMMOCK_URL=\"{}\"\nexport RW_META_ADDR=\"http://{}:{}\"", + cmd.get_args().nth(1).unwrap().to_str().unwrap(), + meta_node.address, + meta_node.port + )); + } + } + Ok("".into()) +} diff --git a/src/rpc_client/src/hummock_meta_client.rs b/src/rpc_client/src/hummock_meta_client.rs index facb14bcb9c49..e7353ef0cab49 100644 --- a/src/rpc_client/src/hummock_meta_client.rs +++ b/src/rpc_client/src/hummock_meta_client.rs @@ -13,10 +13,9 @@ // limitations under the License. use async_trait::async_trait; -use risingwave_hummock_sdk::{HummockEpoch, HummockSSTableId, HummockVersionId}; +use risingwave_hummock_sdk::{HummockEpoch, HummockSSTableId, HummockVersionId, LocalSstableInfo}; use risingwave_pb::hummock::{ - CompactTask, CompactionGroup, HummockVersion, SstableInfo, SubscribeCompactTasksResponse, - VacuumTask, + CompactTask, CompactionGroup, HummockVersion, SubscribeCompactTasksResponse, VacuumTask, }; use tonic::Streaming; @@ -32,9 +31,18 @@ pub trait HummockMetaClient: Send + Sync + 'static { async fn get_new_table_id(&self) -> Result; async fn report_compaction_task(&self, compact_task: CompactTask) -> Result<()>; // We keep `commit_epoch` only for test/benchmark like ssbench. - async fn commit_epoch(&self, epoch: HummockEpoch, sstables: Vec) -> Result<()>; + async fn commit_epoch( + &self, + epoch: HummockEpoch, + sstables: Vec, + ) -> Result<()>; async fn subscribe_compact_tasks(&self) -> Result>; async fn report_vacuum_task(&self, vacuum_task: VacuumTask) -> Result<()>; async fn get_compaction_groups(&self) -> Result>; - async fn trigger_manual_compaction(&self, compaction_group_id: u64) -> Result<()>; + async fn trigger_manual_compaction( + &self, + compaction_group_id: u64, + table_id: u32, + level: u32, + ) -> Result<()>; } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index c150c789b219d..e3aec4e4a5e87 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -18,48 +18,23 @@ use async_trait::async_trait; use paste::paste; use risingwave_common::catalog::{CatalogVersion, TableId}; use risingwave_common::util::addr::HostAddr; -use risingwave_hummock_sdk::{HummockEpoch, HummockSSTableId, HummockVersionId}; +use risingwave_hummock_sdk::{HummockEpoch, HummockSSTableId, HummockVersionId, LocalSstableInfo}; use risingwave_pb::catalog::{ Database as ProstDatabase, Schema as ProstSchema, Source as ProstSource, Table as ProstTable, }; use risingwave_pb::common::{WorkerNode, WorkerType}; use risingwave_pb::ddl_service::ddl_service_client::DdlServiceClient; -use risingwave_pb::ddl_service::{ - CreateDatabaseRequest, CreateDatabaseResponse, CreateMaterializedSourceRequest, - CreateMaterializedSourceResponse, CreateMaterializedViewRequest, - CreateMaterializedViewResponse, CreateSchemaRequest, CreateSchemaResponse, CreateSourceRequest, - CreateSourceResponse, DropDatabaseRequest, DropDatabaseResponse, DropMaterializedSourceRequest, - DropMaterializedSourceResponse, DropMaterializedViewRequest, DropMaterializedViewResponse, - DropSchemaRequest, DropSchemaResponse, DropSourceRequest, DropSourceResponse, -}; +use risingwave_pb::ddl_service::*; use risingwave_pb::hummock::hummock_manager_service_client::HummockManagerServiceClient; -use risingwave_pb::hummock::{ - CompactTask, CompactionGroup, GetCompactionGroupsRequest, GetCompactionGroupsResponse, - GetNewTableIdRequest, GetNewTableIdResponse, HummockSnapshot, HummockVersion, - PinSnapshotRequest, PinSnapshotResponse, PinVersionRequest, PinVersionResponse, - ReportCompactionTasksRequest, ReportCompactionTasksResponse, ReportVacuumTaskRequest, - ReportVacuumTaskResponse, SstableInfo, SubscribeCompactTasksRequest, - SubscribeCompactTasksResponse, TriggerManualCompactionRequest, TriggerManualCompactionResponse, - UnpinSnapshotBeforeRequest, UnpinSnapshotBeforeResponse, UnpinSnapshotRequest, - UnpinSnapshotResponse, UnpinVersionRequest, UnpinVersionResponse, VacuumTask, -}; +use risingwave_pb::hummock::*; use risingwave_pb::meta::cluster_service_client::ClusterServiceClient; use risingwave_pb::meta::heartbeat_service_client::HeartbeatServiceClient; use risingwave_pb::meta::notification_service_client::NotificationServiceClient; use risingwave_pb::meta::stream_manager_service_client::StreamManagerServiceClient; -use risingwave_pb::meta::{ - ActivateWorkerNodeRequest, ActivateWorkerNodeResponse, AddWorkerNodeRequest, - AddWorkerNodeResponse, DeleteWorkerNodeRequest, DeleteWorkerNodeResponse, FlushRequest, - FlushResponse, HeartbeatRequest, HeartbeatResponse, ListAllNodesRequest, ListAllNodesResponse, - SubscribeRequest, SubscribeResponse, -}; +use risingwave_pb::meta::*; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_pb::user::user_service_client::UserServiceClient; -use risingwave_pb::user::{ - CreateUserRequest, CreateUserResponse, DropUserRequest, DropUserResponse, GrantPrivilege, - GrantPrivilegeRequest, GrantPrivilegeResponse, RevokePrivilegeRequest, RevokePrivilegeResponse, - UserInfo, -}; +use risingwave_pb::user::*; use tokio::sync::mpsc::Receiver; use tokio::sync::oneshot::Sender; use tokio::task::JoinHandle; @@ -349,6 +324,12 @@ impl MetaClient { (join_handle, shutdown_tx) } + pub async fn list_materialize_view(&self) -> Result> { + let request = ListMaterializedViewRequest {}; + let resp = self.inner.list_materialized_view(request).await?; + Ok(resp.tables) + } + pub async fn flush(&self) -> Result<()> { let request = FlushRequest::default(); self.inner.flush(request).await?; @@ -424,7 +405,11 @@ impl HummockMetaClient for MetaClient { Ok(()) } - async fn commit_epoch(&self, _epoch: HummockEpoch, _sstables: Vec) -> Result<()> { + async fn commit_epoch( + &self, + _epoch: HummockEpoch, + _sstables: Vec, + ) -> Result<()> { panic!("Only meta service can commit_epoch in production.") } @@ -449,9 +434,19 @@ impl HummockMetaClient for MetaClient { Ok(resp.compaction_groups) } - async fn trigger_manual_compaction(&self, compaction_group_id: u64) -> Result<()> { + async fn trigger_manual_compaction( + &self, + compaction_group_id: u64, + table_id: u32, + level: u32, + ) -> Result<()> { + // TODO: support key_range parameter let req = TriggerManualCompactionRequest { compaction_group_id, + table_id, /* if table_id not exist, manual_compaction will include all the sst + * without check internal_table_id */ + level, + ..Default::default() }; self.inner.trigger_manual_compaction(req).await?; @@ -555,6 +550,7 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, drop_source, DropSourceRequest, DropSourceResponse } ,{ ddl_client, drop_database, DropDatabaseRequest, DropDatabaseResponse } ,{ ddl_client, drop_schema, DropSchemaRequest, DropSchemaResponse } + ,{ ddl_client, list_materialized_view, ListMaterializedViewRequest, ListMaterializedViewResponse } ,{ hummock_client, pin_version, PinVersionRequest, PinVersionResponse } ,{ hummock_client, unpin_version, UnpinVersionRequest, UnpinVersionResponse } ,{ hummock_client, pin_snapshot, PinSnapshotRequest, PinSnapshotResponse } diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index f3df7fb87e561..011cbfbfcc62e 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -27,6 +27,7 @@ lz4 = "1.23.1" madsim = "=0.2.0-alpha.3" memcomparable = { path = "../utils/memcomparable" } moka = { version = "0.8", features = ["future"] } +nix = { version = "0.24.1", features = ["fs"] } num-integer = "0.1" num-traits = "0.2" parking_lot = "0.12" diff --git a/src/storage/benches/bench_merge_iter.rs b/src/storage/benches/bench_merge_iter.rs index 9f8acc0ec9f4b..15cab6810ad72 100644 --- a/src/storage/benches/bench_merge_iter.rs +++ b/src/storage/benches/bench_merge_iter.rs @@ -19,6 +19,7 @@ use std::sync::Arc; use bytes::Bytes; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; use futures::executor::block_on; +use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_storage::hummock::iterator::{ BoxedForwardHummockIterator, Forward, HummockIterator, MergeIterator, OrderedAwareMergeIterator, }; @@ -40,7 +41,12 @@ fn gen_interleave_shared_buffer_batch_iter( HummockValue::put(Bytes::copy_from_slice("value".as_bytes())), )); } - let batch = SharedBufferBatch::new(batch_data, 2333, buffer_tracker.clone()); + let batch = SharedBufferBatch::new( + batch_data, + 2333, + buffer_tracker.clone(), + StaticCompactionGroupId::StateDefault.into(), + ); iterators.push(Box::new(batch.into_forward_iter()) as BoxedForwardHummockIterator); } iterators diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs new file mode 100644 index 0000000000000..093b6fbc42205 --- /dev/null +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -0,0 +1,46 @@ +// Copyright 2022 Singularity Data +// +// 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::hummock::{HummockVersion, Level}; + +use crate::CompactionGroupId; + +pub trait HummockVersionExt { + fn get_compaction_group_levels(&self, compaction_group_id: CompactionGroupId) -> &Vec; + fn get_compaction_group_levels_mut( + &mut self, + compaction_group_id: CompactionGroupId, + ) -> &mut Vec; +} + +impl HummockVersionExt for HummockVersion { + fn get_compaction_group_levels(&self, compaction_group_id: CompactionGroupId) -> &Vec { + &self + .levels + .get(&compaction_group_id) + .unwrap_or_else(|| panic!("compaction group {} exists", compaction_group_id)) + .levels + } + + fn get_compaction_group_levels_mut( + &mut self, + compaction_group_id: CompactionGroupId, + ) -> &mut Vec { + &mut self + .levels + .get_mut(&compaction_group_id) + .unwrap_or_else(|| panic!("compaction group {} exists", compaction_group_id)) + .levels + } +} diff --git a/src/storage/hummock_sdk/src/compaction_group/mod.rs b/src/storage/hummock_sdk/src/compaction_group/mod.rs index d41c3c3fae3e4..88246533b441e 100644 --- a/src/storage/hummock_sdk/src/compaction_group/mod.rs +++ b/src/storage/hummock_sdk/src/compaction_group/mod.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod hummock_version_ext; + use std::borrow::Borrow; use std::fmt::{Display, Formatter}; @@ -66,8 +68,6 @@ impl From<&Prefix> for Vec { /// A compaction task's `StaticCompactionGroupId` indicates the compaction group that all its input /// SSTs belong to. pub enum StaticCompactionGroupId { - /// All shared buffer local compaction task goes to here. - SharedBuffer = 1, /// All states goes to here by default. StateDefault = 2, /// All MVs goes to here. diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index c60e77d54bc96..2e84a29dcebfa 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -13,6 +13,8 @@ // limitations under the License. mod version_cmp; + +use risingwave_pb::hummock::SstableInfo; pub use version_cmp::*; pub mod compact; pub mod compaction_group; @@ -33,6 +35,8 @@ pub const FIRST_VERSION_ID: HummockVersionId = 1; pub const LOCAL_SST_ID_MASK: HummockSSTableId = 1 << (HummockSSTableId::BITS - 1); pub const REMOTE_SST_ID_MASK: HummockSSTableId = !LOCAL_SST_ID_MASK; +pub type LocalSstableInfo = (CompactionGroupId, SstableInfo); + pub fn get_remote_sst_id(id: HummockSSTableId) -> HummockSSTableId { id & REMOTE_SST_ID_MASK } diff --git a/src/storage/src/hummock/compactor.rs b/src/storage/src/hummock/compactor.rs index 1dba2a5875623..4c01a205443c2 100644 --- a/src/storage/src/hummock/compactor.rs +++ b/src/storage/src/hummock/compactor.rs @@ -20,7 +20,7 @@ use std::time::{Duration, Instant}; use bytes::{Bytes, BytesMut}; use futures::future::{try_join_all, BoxFuture}; -use futures::{stream, FutureExt, StreamExt}; +use futures::{stream, FutureExt, StreamExt, TryFutureExt}; use itertools::Itertools; use risingwave_common::config::StorageConfig; use risingwave_common::util::compress::decompress_data; @@ -28,8 +28,7 @@ use risingwave_hummock_sdk::compact::compact_task_to_string; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{get_epoch, get_table_id, Epoch, FullKey}; use risingwave_hummock_sdk::key_range::KeyRange; -use risingwave_hummock_sdk::{HummockSSTableId, VersionedComparator}; -use risingwave_pb::common::VNodeBitmap; +use risingwave_hummock_sdk::{CompactionGroupId, HummockSSTableId, VersionedComparator}; use risingwave_pb::hummock::{CompactTask, SstableInfo, SubscribeCompactTasksResponse, VacuumTask}; use risingwave_rpc_client::HummockMetaClient; use tokio::sync::oneshot::Sender; @@ -98,7 +97,6 @@ trait CompactionFilter { } } -#[derive(Clone, Default)] pub struct DummyCompactionFilter; impl CompactionFilter for DummyCompactionFilter {} @@ -108,7 +106,6 @@ pub struct StateCleanUpCompactionFilter { } impl StateCleanUpCompactionFilter { - #[expect(dead_code)] fn new(table_id_set: HashSet) -> Self { StateCleanUpCompactionFilter { existing_table_ids: table_id_set, @@ -138,7 +135,7 @@ pub struct Compactor { compact_task: CompactTask, } -pub type CompactOutput = (usize, Vec<(Sstable, u64, Vec)>); +pub type CompactOutput = (usize, Vec<(Sstable, u64, Vec)>); impl Compactor { /// Create a new compactor. @@ -149,11 +146,58 @@ impl Compactor { } } + /// Flush shared buffer to level0. Resulted SSTs are grouped by compaction group. + pub async fn compact_shared_buffer_by_compaction_group( + context: Arc, + payload: UploadTaskPayload, + ) -> HummockResult)>> { + let mut grouped_payload: HashMap = HashMap::new(); + for uncommitted_list in payload { + let mut next_inner = HashSet::new(); + for uncommitted in uncommitted_list { + let compaction_group_id = match &uncommitted { + UncommittedData::Sst((compaction_group_id, _)) => *compaction_group_id, + UncommittedData::Batch(batch) => batch.compaction_group_id(), + }; + let group = grouped_payload + .entry(compaction_group_id) + .or_insert_with(std::vec::Vec::new); + if !next_inner.contains(&compaction_group_id) { + group.push(vec![]); + next_inner.insert(compaction_group_id); + } + group.last_mut().unwrap().push(uncommitted); + } + } + + let mut futures = vec![]; + for (id, group_payload) in grouped_payload { + let id_copy = id; + futures.push( + Compactor::compact_shared_buffer(context.clone(), group_payload).map_ok( + move |results| { + results + .into_iter() + .map(move |result| (id_copy, result.0, result.1, result.2)) + .collect_vec() + }, + ), + ); + } + // Note that the output is reordered compared with input `payload`. + let result = try_join_all(futures) + .await? + .into_iter() + .flatten() + .collect_vec(); + Ok(result) + } + /// For compaction from shared buffer to level 0, this is the only function gets called. pub async fn compact_shared_buffer( context: Arc, - payload: &UploadTaskPayload, - ) -> HummockResult)>> { + payload: UploadTaskPayload, + ) -> HummockResult)>> { let mut start_user_keys = payload .iter() .flat_map(|data_list| data_list.iter().map(UncommittedData::start_user_key)) @@ -212,7 +256,7 @@ impl Compactor { for (split_index, _) in compact_task.splits.iter().enumerate() { let compactor = compactor.clone(); let iter = build_ordered_merge_iter::( - payload, + &payload, sstable_store.clone(), stats.clone(), &mut local_stats, @@ -392,11 +436,8 @@ impl Compactor { ); } - // TODO #2065: re-enable it after all states are registered correctly. - let compaction_filter = DummyCompactionFilter::default(); - // let compaction_filter = - // StateCleanUpCompactionFilter::new(HashSet::from_iter(compact_task. - // existing_table_ids)); + let compaction_filter = + StateCleanUpCompactionFilter::new(HashSet::from_iter(compact_task.existing_table_ids)); for (split_index, _) in compact_task.splits.iter().enumerate() { let compactor = compactor.clone(); @@ -458,7 +499,7 @@ impl Compactor { .reserve(self.compact_task.splits.len()); let mut compaction_write_bytes = 0; for (_, ssts) in output_ssts { - for (sst, unit_id, vnode_bitmaps) in ssts { + for (sst, unit_id, table_ids) in ssts { let sst_info = SstableInfo { id: sst.id, key_range: Some(risingwave_pb::hummock::KeyRange { @@ -467,7 +508,7 @@ impl Compactor { inf: false, }), file_size: sst.meta.estimated_size as u64, - vnode_bitmaps, + table_ids, unit_id, }; compaction_write_bytes += sst_info.file_size; @@ -559,7 +600,7 @@ impl Compactor { for SealedSstableBuilder { id: table_id, meta, - vnode_bitmaps, + table_ids, upload_join_handle, data_len, unit_id, @@ -567,7 +608,7 @@ impl Compactor { { let sst = Sstable { id: table_id, meta }; let len = data_len; - ssts.push((sst.clone(), unit_id, vnode_bitmaps)); + ssts.push((sst.clone(), unit_id, table_ids)); upload_join_handles.push(upload_join_handle); if self.context.is_share_buffer_compact { diff --git a/src/storage/src/hummock/compactor_tests.rs b/src/storage/src/hummock/compactor_tests.rs index 7e4a8c4c9c83e..ac1f39e2c6083 100644 --- a/src/storage/src/hummock/compactor_tests.rs +++ b/src/storage/src/hummock/compactor_tests.rs @@ -21,6 +21,7 @@ mod tests { use rand::Rng; use risingwave_common::catalog::TableId; use risingwave_common::config::StorageConfig; + use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::get_table_id; use risingwave_meta::hummock::test_utils::setup_compute_env; @@ -131,7 +132,7 @@ mod tests { // 4. get the latest version and check let version = hummock_manager_ref.get_current_version().await; let output_table_id = version - .get_levels() + .get_compaction_group_levels(StaticCompactionGroupId::StateDefault.into()) .last() .unwrap() .table_infos @@ -169,8 +170,6 @@ mod tests { } #[tokio::test] - // TODO #2065: re-enable it after all states are registered correctly. - #[ignore] async fn test_compaction_drop_all_key() { let (_env, hummock_manager_ref, _cluster_manager_ref, worker_node) = setup_compute_env(8080).await; @@ -236,7 +235,10 @@ mod tests { // 4. get the latest version and check let version = hummock_manager_ref.get_current_version().await; - let output_level_info = version.get_levels().last().unwrap(); + let output_level_info = version + .get_compaction_group_levels(StaticCompactionGroupId::StateDefault.into()) + .last() + .unwrap(); assert_eq!(0, output_level_info.total_file_size); // 5. get compact task and there should be none @@ -249,8 +251,6 @@ mod tests { } #[tokio::test] - // TODO #2065: re-enable it after all states are registered correctly. - #[ignore] async fn test_compaction_drop_key_by_existing_table_id() { let (_env, hummock_manager_ref, _cluster_manager_ref, worker_node) = setup_compute_env(8080).await; @@ -326,7 +326,7 @@ mod tests { // 4. get the latest version and check let version: HummockVersion = hummock_manager_ref.get_current_version().await; let table_ids_from_version: Vec<_> = version - .get_levels() + .get_compaction_group_levels(StaticCompactionGroupId::StateDefault.into()) .iter() .flat_map(|level| level.table_infos.iter()) .map(|table_info| table_info.id) diff --git a/src/storage/src/hummock/file_cache/alloc.rs b/src/storage/src/hummock/file_cache/alloc.rs new file mode 100644 index 0000000000000..80ab35d850781 --- /dev/null +++ b/src/storage/src/hummock/file_cache/alloc.rs @@ -0,0 +1,70 @@ +// Copyright 2022 Singularity Data +// +// 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. + +pub struct AlignedAllocator; + +extern crate alloc; +use alloc::alloc::{Allocator, Global}; + +#[inline(always)] +fn align_up(align: usize, v: usize) -> usize { + debug_assert_eq!(align & (align - 1), 0); + (v + align - 1) & !(align - 1) +} + +unsafe impl Allocator for AlignedAllocator { + fn allocate( + &self, + layout: std::alloc::Layout, + ) -> Result, std::alloc::AllocError> { + let layout = + std::alloc::Layout::from_size_align(layout.size(), align_up(ALIGN, layout.align())) + .unwrap(); + Global.allocate(layout) + } + + unsafe fn deallocate(&self, ptr: std::ptr::NonNull, layout: std::alloc::Layout) { + let layout = + std::alloc::Layout::from_size_align(layout.size(), align_up(ALIGN, layout.align())) + .unwrap(); + Global.deallocate(ptr, layout) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + fn assert_alignment(align: usize, v: usize) { + assert_eq!(align & (align - 1), 0); + assert_eq!(v & (align - 1), 0); + } + + #[test] + fn test_aligned_buffer() { + const ALIGN: usize = 512; + let allocator = AlignedAllocator::; + + let mut buf: Vec = Vec::with_capacity_in(ALIGN * 8, &allocator); + assert_alignment(ALIGN, buf.as_ptr().addr()); + + buf.extend_from_slice(&[b'x'; ALIGN * 8]); + assert_alignment(ALIGN, buf.as_ptr().addr()); + assert_eq!(buf, [b'x'; ALIGN * 8]); + + buf.extend_from_slice(&[b'x'; ALIGN * 8]); + assert_alignment(ALIGN, buf.as_ptr().addr()); + assert_eq!(buf, [b'x'; ALIGN * 16]) + } +} diff --git a/src/storage/src/hummock/file_cache/error.rs b/src/storage/src/hummock/file_cache/error.rs new file mode 100644 index 0000000000000..a2fbc8cb0c7bd --- /dev/null +++ b/src/storage/src/hummock/file_cache/error.rs @@ -0,0 +1,27 @@ +// Copyright 2022 Singularity Data +// +// 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. + +#[derive(thiserror::Error, Debug)] +pub enum Error { + #[error("io error: {0}")] + IoError(#[from] std::io::Error), + #[error("nix error: {0}")] + NixError(#[from] nix::errno::Errno), + #[error("unsupported file system, super block magic: {0}")] + UnsupportedFilesystem(i64), + #[error("other error: {0}")] + Other(String), +} + +pub type Result = core::result::Result; diff --git a/src/storage/src/hummock/file_cache/file.rs b/src/storage/src/hummock/file_cache/file.rs new file mode 100644 index 0000000000000..0227b119b4783 --- /dev/null +++ b/src/storage/src/hummock/file_cache/file.rs @@ -0,0 +1,363 @@ +// Copyright 2022 Singularity Data +// +// 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::fs::{File, OpenOptions}; +use std::os::unix::prelude::{AsRawFd, FileExt, OpenOptionsExt, RawFd}; +use std::path::PathBuf; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::Arc; + +use bytes::Buf; +use nix::fcntl::{fallocate, FallocateFlags}; +use nix::sys::stat::fstat; +use nix::unistd::ftruncate; + +use super::error::{Error, Result}; +use super::{asyncify, DioBuffer, DIO_BUFFER_ALLOCATOR, LOGICAL_BLOCK_SIZE}; + +const ST_BLOCK_SIZE: usize = 512; + +const MAGIC: &[u8] = b"hummock-cache-file"; +const VERSION: u32 = 1; + +#[derive(Clone, Debug)] +pub struct CacheFileOptions { + pub dir: String, + pub id: u64, + + pub fs_block_size: usize, + /// NOTE: `block_size` must be a multiple of `fs_block_size`. + pub block_size: usize, + pub meta_blocks: usize, + pub fallocate_unit: usize, +} + +impl CacheFileOptions { + fn assert(&self) { + assert_pow2(LOGICAL_BLOCK_SIZE); + assert_alignment(LOGICAL_BLOCK_SIZE, self.fs_block_size); + assert_alignment(self.fs_block_size, self.block_size); + } +} + +struct CacheFileCore { + file: std::fs::File, + len: AtomicUsize, + capacity: AtomicUsize, +} + +/// # Format +/// +/// ```plain +/// header block (1 bs, < logical block size used) +/// +/// | MAGIC | version | block size | meta blocks | +/// +/// meta blocks ({meta blocks} bs) +/// +/// | slot 0 index | slot 1 index | ... | padding | (1 bs) +/// | slot i index | slot i + 1 index | ... | padding | (1 bs) +/// ... +/// +/// data blocks +/// +/// | slot 0 data | slot 1 data | ... | +/// ``` +#[derive(Clone)] +pub struct CacheFile { + dir: String, + id: u64, + + pub fs_block_size: usize, + pub block_size: usize, + pub meta_blocks: usize, + pub fallocate_unit: usize, + + core: Arc, +} + +impl std::fmt::Debug for CacheFile { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("CacheFile") + .field( + "path", + &PathBuf::from(self.dir.as_str()).join(filename(self.id)), + ) + .finish() + } +} + +impl CacheFile { + /// Opens the cache file. + /// + /// The underlying file is opened with `O_DIRECT` flag. All I/O requests must be aligned with + /// the logical block size. Additionally, [`CacheFile`] requires I/O size must be a multipler of + /// `options.block_size` (which is required to be a multipler of the file system block size). + /// With this restriction, blocks can be directly reclaimed by the file system after hole + /// punching. + /// + /// Steps: + /// + /// 1. open the underlying file + /// 2. (a) write header block if newly created + /// (b) read header block if exists + /// 3. read meta blocks to [`DioBuffer`] (TODO) + /// 4. pre-allocate space + pub async fn open(options: CacheFileOptions) -> Result { + options.assert(); + + // 1. + let path = PathBuf::from(options.dir.as_str()).join(filename(options.id)); + let mut oopts = OpenOptions::new(); + oopts.create(true); + oopts.read(true); + oopts.write(true); + oopts.custom_flags(libc::O_DIRECT); + + let (file, block_size, meta_blocks, len, capacity, _buffer) = asyncify(move || { + let file = oopts.open(path)?; + let fd = file.as_raw_fd(); + let stat = fstat(fd)?; + if stat.st_blocks == 0 { + // 2a. + write_header(&file, options.block_size, options.meta_blocks)?; + // 3. + let meta_len = options.block_size * options.meta_blocks; + let mut buffer = DioBuffer::with_capacity_in(meta_len, &DIO_BUFFER_ALLOCATOR); + buffer.resize(meta_len, 0); + ftruncate(fd, (options.block_size * (1 + options.meta_blocks)) as i64)?; + // 4. + fallocate( + fd, + FallocateFlags::FALLOC_FL_KEEP_SIZE, + 0, + options.fallocate_unit as i64, + )?; + Ok(( + file, + options.block_size, + options.meta_blocks, + (options.block_size * (1 + options.meta_blocks)) as usize, + options.fallocate_unit, + buffer, + )) + } else { + // 2b. + let (block_size, meta_blocks) = read_header(&file)?; + // 3. + let meta_len = options.block_size * options.meta_blocks; + let mut buffer = + DioBuffer::with_capacity_in(block_size * meta_blocks, &DIO_BUFFER_ALLOCATOR); + buffer.resize(meta_len, 0); + file.read_exact_at(&mut buffer, block_size as u64)?; + // 4. + fallocate( + fd, + FallocateFlags::FALLOC_FL_KEEP_SIZE, + stat.st_size as i64, + options.fallocate_unit as i64, + )?; + Ok(( + file, + block_size, + meta_blocks, + stat.st_size as usize, + stat.st_size as usize + options.fallocate_unit, + buffer, + )) + } + }) + .await?; + + Ok(Self { + dir: options.dir, + id: options.id, + + fs_block_size: options.fs_block_size, + block_size, + meta_blocks, + fallocate_unit: options.fallocate_unit, + + core: Arc::new(CacheFileCore { + file, + len: AtomicUsize::new(len), + capacity: AtomicUsize::new(capacity), + }), + }) + } + + pub async fn append(&self) -> Result<()> { + todo!() + } + + pub async fn write(&self) -> Result<()> { + todo!() + } + + pub async fn read(&self) -> Result<()> { + todo!() + } + + pub async fn flush(&self) -> Result<()> { + todo!() + } + + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Get file length in bytes. + /// + /// `len()` stands for the last written byte of the file. + pub fn len(&self) -> usize { + self.core.len.load(Ordering::Acquire) + } + + /// Get file pre-allocated length in bytes. + /// + /// `capacity()` stands for the last pre-allocated byte of the file. + pub fn capacity(&self) -> usize { + self.core.capacity.load(Ordering::Acquire) + } + + /// Get file size by `stat.st_blocks * FS_BLOCK_SIZE`. + /// + /// `size()` stands for how much space that the file really used. + /// + /// `size()` can be different from `len()` because the file is sparse and pre-allocated. + pub fn size(&self) -> usize { + fstat(self.fd()).unwrap().st_blocks as usize * ST_BLOCK_SIZE + } + + pub fn block_size(&self) -> usize { + self.block_size + } + + pub fn meta_blocks(&self) -> usize { + self.meta_blocks + } +} + +impl CacheFile { + #[inline(always)] + fn fd(&self) -> RawFd { + self.core.file.as_raw_fd() + } +} + +#[inline(always)] +fn filename(id: u64) -> String { + format!("cf-{:020}", id) +} + +fn write_header(file: &File, block_size: usize, meta_blocks: usize) -> Result<()> { + let mut buf: DioBuffer = Vec::with_capacity_in(LOGICAL_BLOCK_SIZE, &DIO_BUFFER_ALLOCATOR); + + buf.extend_from_slice(MAGIC); + buf.extend_from_slice(&VERSION.to_be_bytes()); + buf.extend_from_slice(&block_size.to_be_bytes()); + buf.extend_from_slice(&meta_blocks.to_be_bytes()); + buf.resize(LOGICAL_BLOCK_SIZE, 0); + + file.write_all_at(&buf, 0)?; + Ok(()) +} + +fn read_header(file: &File) -> Result<(usize, usize)> { + let mut buf: DioBuffer = Vec::with_capacity_in(LOGICAL_BLOCK_SIZE, &DIO_BUFFER_ALLOCATOR); + buf.resize(LOGICAL_BLOCK_SIZE, 0); + file.read_exact_at(&mut buf, 0)?; + let mut cursor = 0; + + cursor += MAGIC.len(); + let magic = &buf[cursor - MAGIC.len()..cursor]; + if magic != MAGIC { + return Err(Error::Other(format!( + "magic mismatch, expected: {:?}, got: {:?}", + MAGIC, magic + ))); + } + + cursor += 4; + let version = (&buf[cursor - 4..cursor]).get_u32(); + if version != VERSION { + return Err(Error::Other(format!("unsupported version: {}", version))); + } + + cursor += 8; + let block_size = (&buf[cursor - 8..cursor]).get_u64() as usize; + + cursor += 8; + let meta_blocks = (&buf[cursor - 8..cursor]).get_u64() as usize; + + Ok((block_size, meta_blocks)) +} + +#[inline(always)] +fn assert_pow2(v: usize) { + assert_eq!(v & (v - 1), 0); +} + +#[inline(always)] +fn assert_alignment(align: usize, v: usize) { + assert_eq!(v & (align - 1), 0, "align: {}, v: {}", align, v); +} + +#[inline(always)] +fn _align_up(align: usize, v: usize) -> usize { + (v + align - 1) & !(align - 1) +} + +#[inline(always)] +fn _align_down(align: usize, v: usize) -> usize { + v & !(align - 1) +} + +#[cfg(test)] +mod tests { + use super::*; + + fn is_send_sync_clone() {} + + #[test] + fn ensure_send_sync_clone() { + is_send_sync_clone::(); + } + + #[tokio::test] + async fn test_file_cache() { + let tempdir = tempfile::tempdir().unwrap(); + let options = CacheFileOptions { + dir: tempdir.path().to_str().unwrap().to_string(), + id: 1, + + fs_block_size: 4096, + block_size: 4096, + meta_blocks: 64, + fallocate_unit: 64 * 1024 * 1024, + }; + let cf = CacheFile::open(options.clone()).await.unwrap(); + assert_eq!(cf.block_size, 4096); + assert_eq!(cf.meta_blocks, 64); + assert_eq!(cf.len(), 4096 * 65); + assert_eq!(cf.size(), 64 * 1024 * 1024); + drop(cf); + + let cf = CacheFile::open(options).await.unwrap(); + assert_eq!(cf.block_size, 4096); + assert_eq!(cf.meta_blocks, 64); + assert_eq!(cf.len(), 4096 * 65); + assert_eq!(cf.size(), 64 * 1024 * 1024 + 4096 * 65); + } +} diff --git a/src/storage/src/hummock/file_cache/manager.rs b/src/storage/src/hummock/file_cache/manager.rs new file mode 100644 index 0000000000000..c3bcafc4007c9 --- /dev/null +++ b/src/storage/src/hummock/file_cache/manager.rs @@ -0,0 +1,86 @@ +// Copyright 2022 Singularity Data +// +// 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::path::PathBuf; + +use nix::sys::statfs::{statfs, FsType as NixFsType, EXT4_SUPER_MAGIC}; + +use super::error::{Error, Result}; + +#[derive(Clone, Copy, Debug)] +pub enum FsType { + Ext4, + Xfs, +} + +#[derive(Debug)] +pub struct FileCacheManagerOptions { + pub dir: String, +} + +pub struct FileCacheManager { + _dir: String, + + _fs_type: FsType, + _fs_block_size: usize, +} + +impl FileCacheManager { + pub async fn open(options: FileCacheManagerOptions) -> Result { + if !PathBuf::from(options.dir.as_str()).exists() { + std::fs::create_dir_all(options.dir.as_str())?; + } + + // Get file system type and block size by `statfs(2)`. + let fs_stat = statfs(options.dir.as_str())?; + let fs_type = match fs_stat.filesystem_type() { + EXT4_SUPER_MAGIC => FsType::Ext4, + // FYI: https://github.com/nix-rust/nix/issues/1742 + NixFsType(libc::XFS_SUPER_MAGIC) => FsType::Xfs, + nix_fs_type => return Err(Error::UnsupportedFilesystem(nix_fs_type.0)), + }; + let fs_block_size = fs_stat.block_size() as usize; + + Ok(Self { + _dir: options.dir, + _fs_type: fs_type, + _fs_block_size: fs_block_size, + }) + } +} + +#[cfg(test)] +mod tests { + + use super::*; + + #[tokio::test] + async fn test_file_cache_manager() { + let ci: bool = std::env::var("RISINGWAVE_CI") + .unwrap_or_else(|_| "false".to_string()) + .parse() + .expect("env $RISINGWAVE_CI must be 'true' or 'false'"); + + let tempdir = if ci { + tempfile::Builder::new().tempdir_in("/risingwave").unwrap() + } else { + tempfile::tempdir().unwrap() + }; + + let options = FileCacheManagerOptions { + dir: tempdir.path().to_str().unwrap().to_string(), + }; + let _manager = FileCacheManager::open(options).await.unwrap(); + } +} diff --git a/src/storage/src/hummock/file_cache/mod.rs b/src/storage/src/hummock/file_cache/mod.rs new file mode 100644 index 0000000000000..b12e33ceedc8b --- /dev/null +++ b/src/storage/src/hummock/file_cache/mod.rs @@ -0,0 +1,43 @@ +// Copyright 2022 Singularity Data +// +// 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. + +pub mod alloc; +pub mod error; +pub mod file; +pub mod manager; + +async fn asyncify(f: F) -> error::Result +where + F: FnOnce() -> error::Result + Send + 'static, + T: Send + 'static, +{ + match tokio::task::spawn_blocking(f).await { + Ok(res) => res, + Err(_) => Err(error::Error::Other("background task failed".to_string())), + } +} + +/// The logical block size of the underlying storage (typically 512 bytes). +/// +/// Can be determined using `ioctl(2)` `BLKSSZGET` operation or from the sheel using the command: +/// +/// ```bash +/// blockdev --getss +/// ``` +/// +/// For more details, see man open(2) NOTES section. +const LOGICAL_BLOCK_SIZE: usize = 512; +type DioBuffer = Vec>; +static DIO_BUFFER_ALLOCATOR: alloc::AlignedAllocator = + alloc::AlignedAllocator::; diff --git a/src/storage/src/hummock/hummock_meta_client.rs b/src/storage/src/hummock/hummock_meta_client.rs index 90295d4d5bc02..da4d17167a231 100644 --- a/src/storage/src/hummock/hummock_meta_client.rs +++ b/src/storage/src/hummock/hummock_meta_client.rs @@ -15,9 +15,9 @@ use std::sync::Arc; use async_trait::async_trait; +use risingwave_hummock_sdk::LocalSstableInfo; use risingwave_pb::hummock::{ - CompactTask, CompactionGroup, HummockVersion, SstableInfo, SubscribeCompactTasksResponse, - VacuumTask, + CompactTask, CompactionGroup, HummockVersion, SubscribeCompactTasksResponse, VacuumTask, }; use risingwave_rpc_client::error::Result; use risingwave_rpc_client::{HummockMetaClient, MetaClient}; @@ -92,7 +92,11 @@ impl HummockMetaClient for MonitoredHummockMetaClient { res } - async fn commit_epoch(&self, _epoch: HummockEpoch, _sstables: Vec) -> Result<()> { + async fn commit_epoch( + &self, + _epoch: HummockEpoch, + _sstables: Vec, + ) -> Result<()> { panic!("Only meta service can commit_epoch in production.") } @@ -108,9 +112,14 @@ impl HummockMetaClient for MonitoredHummockMetaClient { self.meta_client.get_compaction_groups().await } - async fn trigger_manual_compaction(&self, compaction_group_id: u64) -> Result<()> { + async fn trigger_manual_compaction( + &self, + compaction_group_id: u64, + table_id: u32, + level: u32, + ) -> Result<()> { self.meta_client - .trigger_manual_compaction(compaction_group_id) + .trigger_manual_compaction(compaction_group_id, table_id, level) .await } } diff --git a/src/storage/src/hummock/local_version.rs b/src/storage/src/hummock/local_version.rs index b81b3acb3626b..3136c8e4db5b1 100644 --- a/src/storage/src/hummock/local_version.rs +++ b/src/storage/src/hummock/local_version.rs @@ -17,6 +17,8 @@ use std::sync::Arc; use parking_lot::lock_api::ArcRwLockReadGuard; use parking_lot::{RawRwLock, RwLock}; +use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; +use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::{HummockEpoch, HummockVersionId}; use risingwave_pb::hummock::{HummockVersion, Level}; use tokio::sync::mpsc::UnboundedSender; @@ -132,7 +134,9 @@ impl PinnedVersion { } pub fn levels(&self) -> &Vec { - &self.version.levels + // TODO #2065: use correct compaction group id + self.version + .get_compaction_group_levels(StaticCompactionGroupId::StateDefault.into()) } pub fn max_committed_epoch(&self) -> u64 { diff --git a/src/storage/src/hummock/local_version_manager.rs b/src/storage/src/hummock/local_version_manager.rs index 6f451764e3b34..5f29ca7c5277d 100644 --- a/src/storage/src/hummock/local_version_manager.rs +++ b/src/storage/src/hummock/local_version_manager.rs @@ -21,8 +21,10 @@ use bytes::Bytes; use itertools::Itertools; use parking_lot::RwLock; use risingwave_common::config::StorageConfig; +use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::FullKey; -use risingwave_pb::hummock::{HummockVersion, SstableInfo}; +use risingwave_hummock_sdk::LocalSstableInfo; +use risingwave_pb::hummock::HummockVersion; use risingwave_rpc_client::HummockMetaClient; use tokio::sync::mpsc::error::TryRecvError; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; @@ -159,9 +161,11 @@ impl LocalVersionManager { /// being referenced by some readers. pub fn try_update_pinned_version(&self, newly_pinned_version: HummockVersion) -> bool { let new_version_id = newly_pinned_version.id; - if validate_table_key_range(&newly_pinned_version.levels).is_err() { - error!("invalid table key range: {:?}", newly_pinned_version.levels); - return false; + for levels in newly_pinned_version.levels.values() { + if validate_table_key_range(&levels.levels).is_err() { + error!("invalid table key range: {:?}", levels.levels); + return false; + } } let mut guard = self.local_version.write(); @@ -238,6 +242,7 @@ impl LocalVersionManager { } } + // TODO #2065: use correct compaction group id let batch = SharedBufferBatch::new_with_size( sorted_items, epoch, @@ -247,6 +252,7 @@ impl LocalVersionManager { } else { self.buffer_tracker.upload_size.clone() }, + StaticCompactionGroupId::StateDefault.into(), ); // Try get shared buffer with version read lock @@ -386,7 +392,7 @@ impl LocalVersionManager { self.local_version.read().pinned_version().clone() } - pub fn get_uncommitted_ssts(&self, epoch: HummockEpoch) -> Vec { + pub fn get_uncommitted_ssts(&self, epoch: HummockEpoch) -> Vec { self.local_version .read() .get_shared_buffer(epoch) @@ -572,6 +578,8 @@ mod tests { use std::sync::Arc; use bytes::Bytes; + use itertools::Itertools; + use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_meta::hummock::test_utils::setup_compute_env; use risingwave_meta::hummock::MockHummockMetaClient; use risingwave_pb::hummock::HummockVersion; @@ -712,6 +720,7 @@ mod tests { LocalVersionManager::build_shared_buffer_item_batches(kvs[i].clone(), epochs[i]), epochs[i], buffer_tracker.clone(), + StaticCompactionGroupId::StateDefault.into(), ); assert_eq!( local_version @@ -738,7 +747,10 @@ mod tests { assert_eq!(1, payload[0].len()); assert_eq!(payload[0][0], UncommittedData::Batch(batches[0].clone())); } - shared_buffer_guard.succeed_upload_task(task_id, vec![sst1.clone()]); + shared_buffer_guard.succeed_upload_task( + task_id, + vec![(StaticCompactionGroupId::StateDefault.into(), sst1.clone())], + ); } let local_version = local_version_manager.get_local_version(); @@ -768,7 +780,10 @@ mod tests { .get_shared_buffer(epochs[0]) .unwrap() .read() - .get_ssts_to_commit(); + .get_ssts_to_commit() + .into_iter() + .map(|(_, sst)| sst) + .collect_vec(); assert_eq!(epoch_uncommitted_ssts.len(), 1); assert_eq!(*epoch_uncommitted_ssts.first().unwrap(), sst1); @@ -786,7 +801,10 @@ mod tests { assert_eq!(1, payload[0].len()); assert_eq!(payload[0][0], UncommittedData::Batch(batches[1].clone())); } - shared_buffer_guard.succeed_upload_task(task_id, vec![sst2.clone()]); + shared_buffer_guard.succeed_upload_task( + task_id, + vec![(StaticCompactionGroupId::StateDefault.into(), sst2.clone())], + ); } let local_version = local_version_manager.get_local_version(); // Check shared buffer @@ -807,7 +825,10 @@ mod tests { .get_shared_buffer(epochs[1]) .unwrap() .read() - .get_ssts_to_commit(); + .get_ssts_to_commit() + .into_iter() + .map(|(_, sst)| sst) + .collect_vec(); assert_eq!(epoch_uncommitted_ssts.len(), 1); assert_eq!(*epoch_uncommitted_ssts.first().unwrap(), sst2); @@ -837,7 +858,10 @@ mod tests { .get_shared_buffer(epochs[1]) .unwrap() .read() - .get_ssts_to_commit(); + .get_ssts_to_commit() + .into_iter() + .map(|(_, sst)| sst) + .collect_vec(); assert_eq!(epoch_uncommitted_ssts.len(), 1); assert_eq!(*epoch_uncommitted_ssts.first().unwrap(), sst2); diff --git a/src/storage/src/hummock/mod.rs b/src/storage/src/hummock/mod.rs index 7dda6f9a963b9..db0756bdb4e6d 100644 --- a/src/storage/src/hummock/mod.rs +++ b/src/storage/src/hummock/mod.rs @@ -52,6 +52,9 @@ mod utils; mod vacuum; pub mod value; +#[cfg(target_os = "linux")] +pub mod file_cache; + pub use error::*; pub use risingwave_common::cache::{CachableEntry, LookupResult, LruCache}; use value::*; diff --git a/src/storage/src/hummock/shared_buffer/mod.rs b/src/storage/src/hummock/shared_buffer/mod.rs index d9bcbae16d1ce..f4acf3b59ef41 100644 --- a/src/storage/src/hummock/shared_buffer/mod.rs +++ b/src/storage/src/hummock/shared_buffer/mod.rs @@ -22,8 +22,8 @@ use std::ops::{Bound, RangeBounds}; use std::sync::Arc; use itertools::Itertools; -use risingwave_hummock_sdk::is_remote_sst_id; use risingwave_hummock_sdk::key::user_key; +use risingwave_hummock_sdk::{is_remote_sst_id, LocalSstableInfo}; use risingwave_pb::hummock::{KeyRange, SstableInfo}; use self::shared_buffer_batch::SharedBufferBatch; @@ -38,7 +38,7 @@ use crate::monitor::{StateStoreMetrics, StoreLocalStatistic}; #[derive(Debug, Clone, PartialEq)] pub enum UncommittedData { - Sst(SstableInfo), + Sst(LocalSstableInfo), Batch(SharedBufferBatch), } @@ -58,7 +58,7 @@ fn get_sst_key_range(info: &SstableInfo) -> &KeyRange { impl UncommittedData { pub fn start_user_key(&self) -> &[u8] { match self { - UncommittedData::Sst(info) => { + UncommittedData::Sst((_, info)) => { let key_range = get_sst_key_range(info); user_key(key_range.left.as_slice()) } @@ -68,7 +68,7 @@ impl UncommittedData { pub fn end_user_key(&self) -> &[u8] { match self { - UncommittedData::Sst(info) => { + UncommittedData::Sst((_, info)) => { let key_range = get_sst_key_range(info); user_key(key_range.right.as_slice()) } @@ -114,7 +114,7 @@ pub(crate) async fn build_ordered_merge_iter( data_iters.push(Box::new(batch.clone().into_directed_iter::()) as BoxedHummockIterator); } - UncommittedData::Sst(table_info) => { + UncommittedData::Sst((_, table_info)) => { let table = sstable_store.sstable(table_info.id, local_stats).await?; data_iters.push(Box::new(T::SstableIteratorType::create( table, @@ -224,7 +224,7 @@ impl SharedBuffer { UncommittedData::Batch(batch) => { range_overlap(key_range, batch.start_user_key(), batch.end_user_key()) } - UncommittedData::Sst(info) => filter_single_sst(info, key_range), + UncommittedData::Sst((_, info)) => filter_single_sst(info, key_range), }) .map(|((_, order_index), data)| (*order_index, data.clone())); @@ -349,8 +349,8 @@ impl SharedBuffer { pub fn succeed_upload_task( &mut self, order_index: OrderIndex, - new_sst: Vec, - ) -> Vec { + new_sst: Vec, + ) -> Vec { let payload = self .uploading_tasks .remove(&order_index) @@ -388,7 +388,7 @@ impl SharedBuffer { previous_sst } - pub fn get_ssts_to_commit(&self) -> Vec { + pub fn get_ssts_to_commit(&self) -> Vec { assert!( self.uploading_tasks.is_empty(), "when committing sst there should not be uploading task" @@ -399,12 +399,12 @@ impl SharedBuffer { UncommittedData::Batch(_) => { panic!("there should not be any batch when committing sst"); } - UncommittedData::Sst(sst) => { + UncommittedData::Sst((compaction_group_id, sst)) => { assert!( is_remote_sst_id(sst.id), "all sst should be remote when trying to get ssts to commit" ); - ret.push(sst.clone()); + ret.push((*compaction_group_id, sst.clone())); } } } @@ -430,6 +430,7 @@ mod tests { use std::sync::Arc; use bytes::Bytes; + use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{key_with_epoch, user_key}; use super::*; @@ -461,8 +462,12 @@ mod tests { )); } shared_buffer_items.sort_by(|l, r| user_key(&l.0).cmp(&r.0)); - let batch = - SharedBufferBatch::new(shared_buffer_items, epoch, Arc::new(AtomicUsize::new(0))); + let batch = SharedBufferBatch::new( + shared_buffer_items, + epoch, + Arc::new(AtomicUsize::new(0)), + StaticCompactionGroupId::StateDefault.into(), + ); if is_replicate { shared_buffer.replicate_batch(batch.clone()); } else { @@ -591,9 +596,10 @@ mod tests { assert_eq!(payload1[1], vec![UncommittedData::Batch(batch1.clone())]); let sst1 = gen_dummy_sst_info(1, vec![batch1, batch2]); - shared_buffer - .borrow_mut() - .succeed_upload_task(order_index1, vec![sst1.clone()]); + shared_buffer.borrow_mut().succeed_upload_task( + order_index1, + vec![(StaticCompactionGroupId::StateDefault.into(), sst1.clone())], + ); shared_buffer.borrow_mut().fail_upload_task(order_index2); @@ -606,6 +612,12 @@ mod tests { assert_eq!(3, payload3.len()); assert_eq!(vec![UncommittedData::Batch(batch4)], payload3[0]); assert_eq!(vec![UncommittedData::Batch(batch3)], payload3[1]); - assert_eq!(vec![UncommittedData::Sst(sst1)], payload3[2]); + assert_eq!( + vec![UncommittedData::Sst(( + StaticCompactionGroupId::StateDefault.into(), + sst1 + ))], + payload3[2] + ); } } diff --git a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs index db4359957604d..07edb8c916337 100644 --- a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs +++ b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs @@ -21,6 +21,7 @@ use std::sync::Arc; use async_trait::async_trait; use bytes::Bytes; +use risingwave_hummock_sdk::CompactionGroupId; use crate::hummock::iterator::{ Backward, DirectionEnum, Forward, HummockIterator, HummockIteratorDirection, @@ -72,6 +73,7 @@ impl PartialEq for SharedBufferBatchInner { pub struct SharedBufferBatch { inner: Arc, epoch: HummockEpoch, + compaction_group_id: CompactionGroupId, } impl SharedBufferBatch { @@ -79,9 +81,16 @@ impl SharedBufferBatch { sorted_items: Vec, epoch: HummockEpoch, buffer_size_tracker: Arc, + compaction_group_id: CompactionGroupId, ) -> Self { let size: usize = Self::measure_batch_size(&sorted_items); - Self::new_with_size(sorted_items, epoch, size, buffer_size_tracker) + Self::new_with_size( + sorted_items, + epoch, + size, + buffer_size_tracker, + compaction_group_id, + ) } pub fn new_with_size( @@ -89,6 +98,7 @@ impl SharedBufferBatch { epoch: HummockEpoch, size: usize, buffer_size_tracker: Arc, + compaction_group_id: CompactionGroupId, ) -> Self { buffer_size_tracker.fetch_add(size, Relaxed); @@ -99,6 +109,7 @@ impl SharedBufferBatch { buffer_size_tracker, }), epoch, + compaction_group_id, } } @@ -168,6 +179,10 @@ impl SharedBufferBatch { pub fn size(&self) -> usize { self.inner.size } + + pub fn compaction_group_id(&self) -> CompactionGroupId { + self.compaction_group_id + } } pub struct SharedBufferBatchIterator { @@ -273,6 +288,7 @@ impl HummockIterator for SharedBufferBatchIterator< mod tests { use itertools::Itertools; + use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::user_key; use super::*; @@ -309,6 +325,7 @@ mod tests { transform_shared_buffer(shared_buffer_items.clone()), epoch, buffer_size_tracker, + StaticCompactionGroupId::StateDefault.into(), ); // Sketch @@ -386,6 +403,7 @@ mod tests { transform_shared_buffer(shared_buffer_items.clone()), epoch, buffer_size_tracker, + StaticCompactionGroupId::StateDefault.into(), ); // FORWARD: Seek to a key < 1st key, expect all three items to return diff --git a/src/storage/src/hummock/shared_buffer/shared_buffer_uploader.rs b/src/storage/src/hummock/shared_buffer/shared_buffer_uploader.rs index 0a671481c47e0..85f61101ca0b0 100644 --- a/src/storage/src/hummock/shared_buffer/shared_buffer_uploader.rs +++ b/src/storage/src/hummock/shared_buffer/shared_buffer_uploader.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use futures::{FutureExt, StreamExt}; use risingwave_common::config::StorageConfig; -use risingwave_hummock_sdk::{get_local_sst_id, HummockEpoch}; +use risingwave_hummock_sdk::{get_local_sst_id, HummockEpoch, LocalSstableInfo}; use risingwave_pb::hummock::SstableInfo; use risingwave_rpc_client::HummockMetaClient; use tokio::sync::{mpsc, oneshot}; @@ -34,7 +34,7 @@ use crate::monitor::StateStoreMetrics; pub(crate) type UploadTaskPayload = OrderSortedUncommittedData; pub(crate) type UploadTaskResult = - BTreeMap<(HummockEpoch, OrderIndex), HummockResult>>; + BTreeMap<(HummockEpoch, OrderIndex), HummockResult>>; #[derive(Debug)] pub struct UploadTask { @@ -146,12 +146,10 @@ impl SharedBufferUploader { "failed due to previous failure", )) } else { - self.flush(epoch, is_local, &payload) - .await - .inspect_err(|e| { - error!("Failed to flush shared buffer: {:?}", e); - failed = true; - }) + self.flush(epoch, is_local, payload).await.inspect_err(|e| { + error!("Failed to flush shared buffer: {:?}", e); + failed = true; + }) }; assert!( task_results.insert((epoch, order_index), result).is_none(), @@ -170,8 +168,8 @@ impl SharedBufferUploader { &self, _epoch: HummockEpoch, is_local: bool, - payload: &UploadTaskPayload, - ) -> HummockResult> { + payload: UploadTaskPayload, + ) -> HummockResult> { if payload.is_empty() { return Ok(vec![]); } @@ -198,20 +196,29 @@ impl SharedBufferUploader { compaction_executor: self.compaction_executor.as_ref().cloned(), }; - let tables = Compactor::compact_shared_buffer(Arc::new(mem_compactor_ctx), payload).await?; + let tables = Compactor::compact_shared_buffer_by_compaction_group( + Arc::new(mem_compactor_ctx), + payload, + ) + .await?; - let uploaded_sst_info: Vec = tables + let uploaded_sst_info = tables .into_iter() - .map(|(sst, unit_id, vnode_bitmaps)| SstableInfo { - id: sst.id, - key_range: Some(risingwave_pb::hummock::KeyRange { - left: sst.meta.smallest_key.clone(), - right: sst.meta.largest_key.clone(), - inf: false, - }), - file_size: sst.meta.estimated_size as u64, - vnode_bitmaps, - unit_id, + .map(|(compaction_group_id, sst, unit_id, table_ids)| { + ( + compaction_group_id, + SstableInfo { + id: sst.id, + key_range: Some(risingwave_pb::hummock::KeyRange { + left: sst.meta.smallest_key.clone(), + right: sst.meta.largest_key.clone(), + inf: false, + }), + file_size: sst.meta.estimated_size as u64, + table_ids, + unit_id, + }, + ) }) .collect(); diff --git a/src/storage/src/hummock/sstable/builder.rs b/src/storage/src/hummock/sstable/builder.rs index 66962ab92b2f5..0b8cf45dffb4a 100644 --- a/src/storage/src/hummock/sstable/builder.rs +++ b/src/storage/src/hummock/sstable/builder.rs @@ -12,13 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeMap; +use std::collections::BTreeSet; use bytes::{BufMut, Bytes, BytesMut}; use risingwave_common::config::StorageConfig; -use risingwave_common::hash::{VNODE_BITMAP_LEN, VNODE_BITS}; use risingwave_hummock_sdk::key::{get_table_id, user_key}; -use risingwave_pb::common::VNodeBitmap; use super::bloom::Bloom; use super::utils::CompressionAlgorithm; @@ -79,8 +77,8 @@ pub struct SSTableBuilder { block_builder: Option, /// Block metadata vec. block_metas: Vec, - /// `table_id` -> Bitmaps of value meta. - vnode_bitmaps: BTreeMap, + /// `table_id` of added keys. + table_ids: BTreeSet, /// Hashes of user keys. user_key_hashes: Vec, /// Last added full key. @@ -96,7 +94,7 @@ impl SSTableBuilder { buf: BytesMut::with_capacity(options.capacity), block_builder: None, block_metas: Vec::with_capacity(options.capacity / options.block_capacity + 1), - vnode_bitmaps: BTreeMap::new(), + table_ids: BTreeSet::new(), user_key_hashes: Vec::with_capacity(options.capacity / DEFAULT_ENTRY_SIZE + 1), last_full_key: Bytes::default(), key_count: 0, @@ -125,14 +123,9 @@ impl SSTableBuilder { // TODO: refine me let mut raw_value = BytesMut::default(); - let value_meta = value.encode(&mut raw_value) & ((1 << VNODE_BITS) - 1); + value.encode(&mut raw_value); if let Some(table_id) = get_table_id(full_key) { - // We use 8 bit of bitmap[x] to indicate existence of virtual node x*8..(x+1)*8, - // respectively - self.vnode_bitmaps - .entry(table_id) - .or_insert([0; VNODE_BITMAP_LEN])[(value_meta >> 3) as usize] |= - 1 << (value_meta & 0b111); + self.table_ids.insert(table_id); } let raw_value = raw_value.freeze(); @@ -164,7 +157,7 @@ impl SSTableBuilder { /// ```plain /// | Block 0 | ... | Block N-1 | N (4B) | /// ``` - pub fn finish(mut self) -> (u64, Bytes, SstableMeta, Vec) { + pub fn finish(mut self) -> (u64, Bytes, SstableMeta, Vec) { let smallest_key = self.block_metas[0].smallest_key.clone(); let largest_key = self.last_full_key.to_vec(); self.build_block(); @@ -192,13 +185,7 @@ impl SSTableBuilder { self.sstable_id, self.buf.freeze(), meta, - self.vnode_bitmaps - .iter() - .map(|(table_id, vnode_bitmaps)| VNodeBitmap { - table_id: *table_id, - bitmap: ::prost::alloc::vec::Vec::from(*vnode_bitmaps), - }) - .collect(), + self.table_ids.into_iter().collect(), ) } diff --git a/src/storage/src/hummock/sstable/group_builder.rs b/src/storage/src/hummock/sstable/group_builder.rs index d4767079b8df4..8a70b03827828 100644 --- a/src/storage/src/hummock/sstable/group_builder.rs +++ b/src/storage/src/hummock/sstable/group_builder.rs @@ -176,7 +176,7 @@ mod tests { use std::sync::atomic::Ordering::SeqCst; use bytes::Buf; - use risingwave_common::hash::VirtualNode; + use risingwave_common::types::VirtualNode; use super::*; use crate::hummock::iterator::test_utils::mock_sstable_store; diff --git a/src/storage/src/hummock/sstable/mod.rs b/src/storage/src/hummock/sstable/mod.rs index 74a2da8db98e0..1896fadbc7e71 100644 --- a/src/storage/src/hummock/sstable/mod.rs +++ b/src/storage/src/hummock/sstable/mod.rs @@ -96,7 +96,7 @@ impl Sstable { inf: false, }), file_size: self.meta.estimated_size as u64, - vnode_bitmaps: vec![], + table_ids: vec![], unit_id: 0, } } diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index 6b1bf502fbd0d..5b4b7a21b661e 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -15,7 +15,6 @@ use futures::Future; use risingwave_hummock_sdk::key::{Epoch, FullKey}; use risingwave_hummock_sdk::HummockSSTableId; -use risingwave_pb::common::VNodeBitmap; use tokio::task::JoinHandle; use super::SstableMeta; @@ -26,7 +25,7 @@ use crate::hummock::{CachePolicy, HummockResult, SSTableBuilder, Sstable}; pub struct SealedSstableBuilder { pub id: HummockSSTableId, pub meta: SstableMeta, - pub vnode_bitmaps: Vec, + pub table_ids: Vec, pub upload_join_handle: JoinHandle>, pub data_len: usize, pub unit_id: u64, @@ -125,7 +124,7 @@ where /// will be no-op. pub fn seal_current(&mut self) { if let Some(builder) = self.current_builder.take() { - let (table_id, data, meta, vnode_bitmap) = builder.finish(); + let (table_id, data, meta, table_ids) = builder.finish(); let len = data.len(); let sstable_store = self.sstable_store.clone(); let meta_clone = meta.clone(); @@ -144,7 +143,7 @@ where self.sealed_builders.push(SealedSstableBuilder { id: table_id, meta, - vnode_bitmaps: vnode_bitmap, + table_ids, upload_join_handle, data_len: len, unit_id: 0, diff --git a/src/storage/src/hummock/state_store.rs b/src/storage/src/hummock/state_store.rs index 80f7406e87527..3d0c288a3ef37 100644 --- a/src/storage/src/hummock/state_store.rs +++ b/src/storage/src/hummock/state_store.rs @@ -20,8 +20,7 @@ use std::sync::Arc; use bytes::Bytes; use itertools::Itertools; use risingwave_hummock_sdk::key::key_with_epoch; -use risingwave_hummock_sdk::HummockEpoch; -use risingwave_pb::hummock::SstableInfo; +use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; use super::iterator::{ BackwardUserIterator, ConcatIteratorInner, DirectedUserIterator, UserIterator, @@ -218,7 +217,7 @@ impl HummockStorage { return Ok(v); } } - UncommittedData::Sst(table_info) => { + UncommittedData::Sst((_, table_info)) => { let table = self .sstable_store .sstable(table_info.id, &mut stats) @@ -414,7 +413,7 @@ impl StateStore for HummockStorage { } } - fn get_uncommitted_ssts(&self, epoch: u64) -> Vec { + fn get_uncommitted_ssts(&self, epoch: u64) -> Vec { self.local_version_manager.get_uncommitted_ssts(epoch) } } diff --git a/src/storage/src/hummock/test_utils.rs b/src/storage/src/hummock/test_utils.rs index c0e09eb83ef3d..df45465ae657c 100644 --- a/src/storage/src/hummock/test_utils.rs +++ b/src/storage/src/hummock/test_utils.rs @@ -23,7 +23,6 @@ use risingwave_hummock_sdk::key::key_with_epoch; use risingwave_hummock_sdk::HummockSSTableId; use risingwave_meta::hummock::test_utils::setup_compute_env; use risingwave_meta::hummock::MockHummockMetaClient; -use risingwave_pb::common::VNodeBitmap; use risingwave_pb::hummock::{KeyRange, SstableInfo}; use super::{CompressionAlgorithm, SstableMeta, DEFAULT_RESTART_INTERVAL}; @@ -83,7 +82,7 @@ pub fn gen_dummy_sst_info(id: HummockSSTableId, batches: Vec) inf: false, }), file_size: batches.len() as u64, - vnode_bitmaps: vec![], + table_ids: vec![], unit_id: u64::MAX, } } @@ -123,13 +122,13 @@ pub fn default_builder_opt_for_test() -> SSTableBuilderOptions { pub fn gen_test_sstable_data( opts: SSTableBuilderOptions, kv_iter: impl Iterator, HummockValue>)>, -) -> (Bytes, SstableMeta, Vec) { +) -> (Bytes, SstableMeta, Vec) { let mut b = SSTableBuilder::new(0, opts); for (key, value) in kv_iter { b.add(&key, value.as_slice()) } - let (_, data, meta, vnodes) = b.finish(); - (data, meta, vnodes) + let (_, data, meta, table_ids) = b.finish(); + (data, meta, table_ids) } /// Generates a test table from the given `kv_iter` and put the kv value to `sstable_store` diff --git a/src/storage/src/hummock/value.rs b/src/storage/src/hummock/value.rs index bc18905c75efc..75ca701ffd7cd 100644 --- a/src/storage/src/hummock/value.rs +++ b/src/storage/src/hummock/value.rs @@ -13,7 +13,7 @@ // limitations under the License. use bytes::{Buf, BufMut, Bytes}; -use risingwave_common::hash::VirtualNode; +use risingwave_common::types::VirtualNode; use super::{HummockError, HummockResult}; use crate::storage_value::{StorageValue, ValueMeta, VALUE_META_SIZE}; diff --git a/src/storage/src/lib.rs b/src/storage/src/lib.rs index 6e42613caa5ff..dbdfd6a654f41 100644 --- a/src/storage/src/lib.rs +++ b/src/storage/src/lib.rs @@ -38,6 +38,9 @@ #![feature(result_option_inspect)] #![feature(generators)] #![feature(lint_reasons)] +#![feature(allocator_api)] +#![feature(strict_provenance)] +#![feature(let_else)] #![test_runner(risingwave_test_runner::test_runner::run_failpont_tests)] pub mod cell_based_row_deserializer; diff --git a/src/storage/src/monitor/monitored_store.rs b/src/storage/src/monitor/monitored_store.rs index a964920b09604..d41bd7fa80056 100644 --- a/src/storage/src/monitor/monitored_store.rs +++ b/src/storage/src/monitor/monitored_store.rs @@ -18,7 +18,7 @@ use std::time::Instant; use bytes::Bytes; use futures::Future; -use risingwave_pb::hummock::SstableInfo; +use risingwave_hummock_sdk::LocalSstableInfo; use tracing::error; use super::StateStoreMetrics; @@ -239,7 +239,7 @@ where } } - fn get_uncommitted_ssts(&self, epoch: u64) -> Vec { + fn get_uncommitted_ssts(&self, epoch: u64) -> Vec { self.inner.get_uncommitted_ssts(epoch) } } diff --git a/src/storage/src/storage_value.rs b/src/storage/src/storage_value.rs index bc9f242be9926..2ea2fc4c9c1aa 100644 --- a/src/storage/src/storage_value.rs +++ b/src/storage/src/storage_value.rs @@ -13,7 +13,7 @@ // limitations under the License. use bytes::{Buf, BufMut, Bytes}; -use risingwave_common::hash::VirtualNode; +use risingwave_common::types::VirtualNode; /// Size of value meta in bytes. Since there might exist paddings between fields in `ValueMeta`, we /// can't simply use `size_of` to retrieve its size. diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index 07e5340f08b80..6fe3457141514 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -16,7 +16,7 @@ use std::ops::RangeBounds; use std::sync::Arc; use bytes::Bytes; -use risingwave_pb::hummock::SstableInfo; +use risingwave_hummock_sdk::LocalSstableInfo; use crate::error::StorageResult; use crate::monitor::{MonitoredStateStore, StateStoreMetrics}; @@ -162,7 +162,7 @@ pub trait StateStore: Send + Sync + 'static + Clone { } /// Gets `epoch`'s uncommitted `SSTables`. - fn get_uncommitted_ssts(&self, _epoch: u64) -> Vec { + fn get_uncommitted_ssts(&self, _epoch: u64) -> Vec { todo!() } } diff --git a/src/storage/src/table/cell_based_table.rs b/src/storage/src/table/cell_based_table.rs index 8be9e44623568..256d49d0bafcf 100644 --- a/src/storage/src/table/cell_based_table.rs +++ b/src/storage/src/table/cell_based_table.rs @@ -17,7 +17,6 @@ use std::ops::Bound::{self, Excluded, Included, Unbounded}; use std::ops::RangeBounds; use std::sync::Arc; -use bytes::Bytes; use futures::{Stream, StreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; @@ -57,7 +56,7 @@ pub struct CellBasedTable { column_descs: Vec, /// Mapping from column id to column index - pk_serializer: Option, + pk_serializer: OrderedRowSerializer, /// Used for serializing the row. cell_based_row_serializer: CellBasedRowSerializer, @@ -88,18 +87,19 @@ impl CellBasedTable { pub fn new( keyspace: Keyspace, column_descs: Vec, - ordered_row_serializer: Option, + order_types: Vec, dist_key_indices: Option>, ) -> Self { let schema = Schema::new(column_descs.iter().map(Into::into).collect_vec()); let column_ids = column_descs.iter().map(|d| d.column_id).collect(); + let pk_serializer = OrderedRowSerializer::new(order_types); Self { keyspace, schema, mapping: Arc::new(make_column_desc_index(column_descs.clone())), column_descs, - pk_serializer: ordered_row_serializer, + pk_serializer, cell_based_row_serializer: CellBasedRowSerializer::new(), column_ids, dist_key_indices, @@ -111,58 +111,49 @@ impl CellBasedTable { column_descs: Vec, order_types: Vec, ) -> Self { - Self::new( - keyspace, - column_descs, - Some(OrderedRowSerializer::new(order_types)), - None, - ) + Self::new(keyspace, column_descs, order_types, None) } pub fn schema(&self) -> &Schema { &self.schema } + pub(super) fn pk_serializer(&self) -> &OrderedRowSerializer { + &self.pk_serializer + } +} + +/// Get & Write +impl CellBasedTable { /// Get a single row by point get pub async fn get_row(&self, pk: &Row, epoch: u64) -> StorageResult> { // TODO: use multi-get for cell_based get_row // TODO: encode vnode into key // let vnode = self.compute_vnode_by_row(pk); - let pk_serializer = self.pk_serializer.as_ref().expect("pk_serializer is None"); - let serialized_pk = serialize_pk(pk, pk_serializer); + let serialized_pk = serialize_pk(pk, &self.pk_serializer); + let sentinel_key = serialize_pk_and_column_id(&serialized_pk, &SENTINEL_CELL_ID).map_err(err)?; - let mut get_res = Vec::new(); - - let sentinel_cell = self.keyspace.get(&sentinel_key, epoch).await?; - - if sentinel_cell.is_none() { + if self.keyspace.get(&sentinel_key, epoch).await?.is_none() { // if sentinel cell is none, this row doesn't exist return Ok(None); - } else { - get_res.push((sentinel_key, sentinel_cell.unwrap())); - } + }; + + let mut row_deserializer = CellBasedRowDeserializer::new(&*self.mapping); for column_id in &self.column_ids { let key = serialize_pk_and_column_id(&serialized_pk, column_id).map_err(err)?; - - let state_store_get_res = self.keyspace.get(&key, epoch).await?; - if let Some(state_store_get_res) = state_store_get_res { - get_res.push((key, state_store_get_res)); + if let Some(value) = self.keyspace.get(&key, epoch).await? { + let deserialize_res = row_deserializer.deserialize(&key, &value).map_err(err)?; + assert!(deserialize_res.is_none()); } } - let mut cell_based_row_deserializer = CellBasedRowDeserializer::new(&*self.mapping); - for (key, value) in get_res { - let deserialize_res = cell_based_row_deserializer - .deserialize(&Bytes::from(key), &value) - .map_err(err)?; - assert!(deserialize_res.is_none()); - } - let pk_and_row = cell_based_row_deserializer.take(); + + let pk_and_row = row_deserializer.take(); Ok(pk_and_row.map(|(_pk, row)| row)) } /// Get vnode value. Should provide a full row (instead of pk). - fn compute_vnode_by_value(&self, value: &Row) -> u16 { + fn compute_vnode_by_row(&self, value: &Row) -> u16 { let dist_key_indices = self.dist_key_indices.as_ref().unwrap(); let hash_builder = CRC32FastBuilder {}; @@ -177,25 +168,21 @@ impl CellBasedTable { // get row by state_store scan // TODO: encode vnode into key // let vnode = self.compute_vnode_by_row(value); - let pk_serializer = self.pk_serializer.as_ref().expect("pk_serializer is None"); - let start_key = serialize_pk(pk, pk_serializer); + let start_key = serialize_pk(pk, &self.pk_serializer); let key_range = range_of_prefix(&start_key); - let state_store_range_scan_res = self + let kv_pairs = self .keyspace .scan_with_range(key_range, None, epoch) .await?; - let mut cell_based_row_deserializer = CellBasedRowDeserializer::new(&*self.mapping); - for (key, value) in state_store_range_scan_res { - cell_based_row_deserializer - .deserialize(&key, &value) - .map_err(err)?; - } - let pk_and_row = cell_based_row_deserializer.take(); - match pk_and_row { - Some(_) => Ok(pk_and_row.map(|(_pk, row)| row)), - None => Ok(None), + + let mut deserializer = CellBasedRowDeserializer::new(&*self.mapping); + for (key, value) in kv_pairs { + deserializer.deserialize(&key, &value).map_err(err)?; } + + let pk_and_row = deserializer.take(); + Ok(pk_and_row.map(|(_pk, row)| row)) } async fn batch_write_rows_inner( @@ -206,7 +193,7 @@ impl CellBasedTable { // stateful executors need to compute vnode. let mut batch = self.keyspace.state_store().start_write_batch(); let mut local = batch.prefixify(&self.keyspace); - let hash_builder = CRC32FastBuilder {}; + for (pk, row_op) in buffer { // If value meta is computed here, then the cell based table is guaranteed to have // distribution keys. Also, it is guaranteed that distribution key indices will @@ -214,8 +201,7 @@ impl CellBasedTable { match row_op { RowOp::Insert(row) => { let value_meta = if WITH_VALUE_META { - let vnode = self.compute_vnode_by_value(&row); - ValueMeta::with_vnode(vnode) + ValueMeta::with_vnode(self.compute_vnode_by_row(&row)) } else { ValueMeta::default() }; @@ -230,11 +216,7 @@ impl CellBasedTable { RowOp::Delete(old_row) => { // TODO(wcy-fdu): only serialize key on deletion let value_meta = if WITH_VALUE_META { - let vnode = old_row - .hash_by_indices(self.dist_key_indices.as_ref().unwrap(), &hash_builder) - .unwrap() - .to_vnode(); - ValueMeta::with_vnode(vnode) + ValueMeta::with_vnode(self.compute_vnode_by_row(&old_row)) } else { ValueMeta::default() }; @@ -248,11 +230,7 @@ impl CellBasedTable { } RowOp::Update((old_row, new_row)) => { let value_meta = if WITH_VALUE_META { - let vnode = new_row - .hash_by_indices(self.dist_key_indices.as_ref().unwrap(), &hash_builder) - .unwrap() - .to_vnode(); - ValueMeta::with_vnode(vnode) + ValueMeta::with_vnode(self.compute_vnode_by_row(&new_row)) } else { ValueMeta::default() }; @@ -327,7 +305,7 @@ impl TableIter for S { } } -/// Iterator functions. +/// Iterators impl CellBasedTable { /// Get a [`StreamingIter`] with given `encoded_key_range`. pub(super) async fn streaming_iter_with_encoded_key_range( @@ -446,15 +424,14 @@ impl CellBasedTable { } } - let pk_serializer = self.pk_serializer.as_ref().expect("pk_serializer is None"); let start_key = serialize_pk_bound( - pk_serializer, + &self.pk_serializer, &pk_prefix, next_col_bounds.start_bound(), true, ); let end_key = serialize_pk_bound( - pk_serializer, + &self.pk_serializer, &pk_prefix, next_col_bounds.end_bound(), false, @@ -475,8 +452,7 @@ impl CellBasedTable { epoch: u64, pk_prefix: Row, ) -> StorageResult> { - let pk_serializer = self.pk_serializer.as_ref().expect("pk_serializer is None"); - let prefix_serializer = pk_serializer.prefix(pk_prefix.size()); + let prefix_serializer = self.pk_serializer.prefix(pk_prefix.size()); let serialized_pk_prefix = serialize_pk(&pk_prefix, &prefix_serializer); let key_range = range_of_prefix(&serialized_pk_prefix); diff --git a/src/storage/src/table/state_table.rs b/src/storage/src/table/state_table.rs index dae477bd00eb9..0bad92f37d2b0 100644 --- a/src/storage/src/table/state_table.rs +++ b/src/storage/src/table/state_table.rs @@ -39,9 +39,6 @@ pub struct StateTable { /// Relation layer cell_based_table: CellBasedTable, - /// Serializer for pk - pk_serializer: OrderedRowSerializer, - pk_indices: Vec, } @@ -53,21 +50,22 @@ impl StateTable { dist_key_indices: Option>, pk_indices: Vec, ) -> Self { - let pk_serializer = OrderedRowSerializer::new(order_types); - Self { mem_table: MemTable::new(), cell_based_table: CellBasedTable::new( keyspace, column_descs, - Some(pk_serializer.clone()), + order_types, dist_key_indices, ), - pk_serializer, pk_indices, } } + fn pk_serializer(&self) -> &OrderedRowSerializer { + self.cell_based_table.pk_serializer() + } + // TODO: remove, should not be exposed to user pub fn get_pk_indices(&self) -> &[usize] { &self.pk_indices @@ -81,8 +79,7 @@ impl StateTable { /// memtable, it will be a [`Cow::Borrowed`]. If is from cell based table, it will be an owned /// value. To convert `Option>` to `Option`, just call `into_owned`. pub async fn get_row(&self, pk: &Row, epoch: u64) -> StorageResult>> { - // TODO: change to Cow to avoid unnecessary clone. - let pk_bytes = serialize_pk(pk, &self.pk_serializer); + let pk_bytes = serialize_pk(pk, self.pk_serializer()); let mem_table_res = self.mem_table.get_row_op(&pk_bytes); match mem_table_res { Some(row_op) => match row_op { @@ -110,7 +107,7 @@ impl StateTable { datums.push(value.index(*pk_index).clone()); } let pk = Row::new(datums); - let pk_bytes = serialize_pk(&pk, &self.pk_serializer); + let pk_bytes = serialize_pk(&pk, self.pk_serializer()); self.mem_table.insert(pk_bytes, value); Ok(()) } @@ -123,7 +120,7 @@ impl StateTable { datums.push(old_value.index(*pk_index).clone()); } let pk = Row::new(datums); - let pk_bytes = serialize_pk(&pk, &self.pk_serializer); + let pk_bytes = serialize_pk(&pk, self.pk_serializer()); self.mem_table.delete(pk_bytes, old_value); Ok(()) } @@ -132,7 +129,7 @@ impl StateTable { pub fn update(&mut self, old_value: Row, new_value: Row) -> StorageResult<()> { let pk = old_value.by_indices(&self.pk_indices); debug_assert_eq!(pk, new_value.by_indices(&self.pk_indices)); - let pk_bytes = serialize_pk(&pk, &self.pk_serializer); + let pk_bytes = serialize_pk(&pk, self.pk_serializer()); self.mem_table.update(pk_bytes, old_value, new_value); Ok(()) } @@ -190,10 +187,10 @@ impl StateTable { { let encoded_start_key = pk_bounds .start_bound() - .map(|pk| serialize_pk(pk.as_ref(), &self.pk_serializer)); + .map(|pk| serialize_pk(pk.as_ref(), self.pk_serializer())); let encoded_end_key = pk_bounds .end_bound() - .map(|pk| serialize_pk(pk.as_ref(), &self.pk_serializer)); + .map(|pk| serialize_pk(pk.as_ref(), self.pk_serializer())); let encoded_key_range = (encoded_start_key, encoded_end_key); self.iter_with_encoded_key_range(encoded_key_range, epoch) @@ -201,13 +198,12 @@ impl StateTable { } /// This function scans rows from the relational table with specific `pk_prefix`. - pub async fn iter_with_pk_prefix<'a>( - &'a self, - pk_prefix: &'a Row, + pub async fn iter_with_pk_prefix( + &self, + pk_prefix: &Row, epoch: u64, - ) -> StorageResult> { - let order_types = &self.pk_serializer.clone().into_order_types()[0..pk_prefix.size()]; - let prefix_serializer = OrderedRowSerializer::new(order_types.into()); + ) -> StorageResult> { + let prefix_serializer = self.pk_serializer().prefix(pk_prefix.size()); let encoded_prefix = serialize_pk(pk_prefix, &prefix_serializer); let encoded_key_range = range_of_prefix(&encoded_prefix); diff --git a/src/stream/Cargo.toml b/src/stream/Cargo.toml index a8b2a27ca24ec..fe06142c9b7c4 100644 --- a/src/stream/Cargo.toml +++ b/src/stream/Cargo.toml @@ -24,6 +24,7 @@ itertools = "0.10" lazy_static = "1" log = "0.4" madsim = "=0.2.0-alpha.3" +maplit = "1.0.2" memcomparable = { path = "../utils/memcomparable" } num-traits = "0.2" parking_lot = "0.12" @@ -35,6 +36,7 @@ rdkafka = { version = "0.28", features = ["cmake-build"] } risingwave_common = { path = "../common" } risingwave_connector = { path = "../connector" } risingwave_expr = { path = "../expr" } +risingwave_hummock_sdk = { path = "../storage/hummock_sdk" } risingwave_pb = { path = "../prost" } risingwave_rpc_client = { path = "../rpc_client" } risingwave_source = { path = "../source" } diff --git a/src/stream/src/executor/aggregation/agg_state.rs b/src/stream/src/executor/aggregation/agg_state.rs index 3c27ade6b30c0..c3fdae121157a 100644 --- a/src/stream/src/executor/aggregation/agg_state.rs +++ b/src/stream/src/executor/aggregation/agg_state.rs @@ -17,6 +17,7 @@ use std::fmt::Debug; use itertools::Itertools; use risingwave_common::array::{ArrayBuilderImpl, Op}; use risingwave_common::types::Datum; +use risingwave_storage::table::state_table::StateTable; use risingwave_storage::StateStore; use crate::executor::error::StreamExecutorResult; @@ -44,9 +45,13 @@ impl Debug for AggState { pub const ROW_COUNT_COLUMN: usize = 0; impl AggState { - pub async fn row_count(&mut self, epoch: u64) -> StreamExecutorResult { + pub async fn row_count( + &mut self, + epoch: u64, + state_table: &StateTable, + ) -> StreamExecutorResult { Ok(self.managed_states[ROW_COUNT_COLUMN] - .get_output(epoch) + .get_output(epoch, state_table) .await? .map(|x| *x.as_int64()) .unwrap_or(0)) @@ -71,14 +76,18 @@ impl AggState { /// changes to the state. If the state is already marked dirty in this epoch, this function does /// no-op. /// After calling this function, `self.is_dirty()` will return `true`. - pub async fn may_mark_as_dirty(&mut self, epoch: u64) -> StreamExecutorResult<()> { + pub async fn may_mark_as_dirty( + &mut self, + epoch: u64, + state_tables: &[StateTable], + ) -> StreamExecutorResult<()> { if self.is_dirty() { return Ok(()); } let mut outputs = vec![]; - for state in &mut self.managed_states { - outputs.push(state.get_output(epoch).await?); + for (state, state_table) in self.managed_states.iter_mut().zip_eq(state_tables.iter()) { + outputs.push(state.get_output(epoch, state_table).await?); } self.prev_states = Some(outputs); Ok(()) @@ -93,12 +102,15 @@ impl AggState { builders: &mut [ArrayBuilderImpl], new_ops: &mut Vec, epoch: u64, + state_tables: &[StateTable], ) -> StreamExecutorResult { if !self.is_dirty() { return Ok(0); } - let row_count = self.row_count(epoch).await?; + let row_count = self + .row_count(epoch, &state_tables[ROW_COUNT_COLUMN]) + .await?; let prev_row_count = self.prev_row_count(); trace!( @@ -120,8 +132,12 @@ impl AggState { // previous state is empty, current state is not empty, insert one `Insert` op. new_ops.push(Op::Insert); - for (builder, state) in builders.iter_mut().zip_eq(self.managed_states.iter_mut()) { - let data = state.get_output(epoch).await?; + for ((builder, state), state_table) in builders + .iter_mut() + .zip_eq(self.managed_states.iter_mut()) + .zip_eq(state_tables.iter()) + { + let data = state.get_output(epoch, state_table).await?; trace!("append_datum (0 -> N): {:?}", &data); builder.append_datum(&data)?; } @@ -149,12 +165,13 @@ impl AggState { new_ops.push(Op::UpdateDelete); new_ops.push(Op::UpdateInsert); - for (builder, prev_state, cur_state) in itertools::multizip(( + for (builder, prev_state, cur_state, state_table) in itertools::multizip(( builders.iter_mut(), self.prev_states.as_ref().unwrap().iter(), self.managed_states.iter_mut(), + state_tables.iter(), )) { - let cur_state = cur_state.get_output(epoch).await?; + let cur_state = cur_state.get_output(epoch, state_table).await?; trace!( "append_datum (N -> N): prev = {:?}, cur = {:?}", prev_state, diff --git a/src/stream/src/executor/aggregation/mod.rs b/src/stream/src/executor/aggregation/mod.rs index 42e5beb7a3013..6ce863ea97bfc 100644 --- a/src/stream/src/executor/aggregation/mod.rs +++ b/src/stream/src/executor/aggregation/mod.rs @@ -470,7 +470,7 @@ pub async fn generate_managed_agg_state( if idx == ROW_COUNT_COLUMN { // For the rowcount state, we should record the rowcount. - let output = managed_state.get_output(epoch).await?; + let output = managed_state.get_output(epoch, &state_tables[idx]).await?; row_count = Some(output.as_ref().map(|x| *x.as_int64() as usize).unwrap_or(0)); } diff --git a/src/stream/src/executor/batch_query.rs b/src/stream/src/executor/batch_query.rs index b00120ebbb817..927239ab95f6a 100644 --- a/src/stream/src/executor/batch_query.rs +++ b/src/stream/src/executor/batch_query.rs @@ -17,7 +17,7 @@ use futures_async_stream::try_stream; use risingwave_common::array::{DataChunk, Op, StreamChunk}; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::{OrderedColumnDesc, Schema}; -use risingwave_common::hash::VIRTUAL_NODE_COUNT; +use risingwave_common::types::VIRTUAL_NODE_COUNT; use risingwave_common::util::hash_util::CRC32FastBuilder; use risingwave_storage::table::cell_based_table::CellBasedTable; use risingwave_storage::table::TableIter; diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs index 983ea434a1aff..4afbe41c54b50 100644 --- a/src/stream/src/executor/dispatch.rs +++ b/src/stream/src/executor/dispatch.rs @@ -24,7 +24,7 @@ use itertools::Itertools; use madsim::collections::{HashMap, HashSet}; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::error::{internal_error, Result}; -use risingwave_common::hash::VIRTUAL_NODE_COUNT; +use risingwave_common::types::VIRTUAL_NODE_COUNT; use risingwave_common::util::addr::{is_local_address, HostAddr}; use risingwave_common::util::hash_util::CRC32FastBuilder; use tracing::event; @@ -779,7 +779,7 @@ mod tests { use risingwave_common::array::stream_chunk::StreamChunkTestExt; use risingwave_common::array::{Array, ArrayBuilder, I32ArrayBuilder, Op}; use risingwave_common::catalog::Schema; - use risingwave_common::hash::VIRTUAL_NODE_COUNT; + use risingwave_common::types::VIRTUAL_NODE_COUNT; use risingwave_pb::common::{ActorInfo, HostAddress}; use super::*; diff --git a/src/stream/src/executor/global_simple_agg.rs b/src/stream/src/executor/global_simple_agg.rs index 10d968bcb237a..b3279e85eb143 100644 --- a/src/stream/src/executor/global_simple_agg.rs +++ b/src/stream/src/executor/global_simple_agg.rs @@ -110,7 +110,7 @@ impl SimpleAggExecutor { ks.clone(), agg_call, &key_indices, - &pk_indices, + &input_info.pk_indices, &schema, input.as_ref(), )); @@ -142,7 +142,7 @@ impl SimpleAggExecutor { keyspace: &[Keyspace], chunk: StreamChunk, epoch: u64, - state_tables: &[StateTable], + state_tables: &mut [StateTable], ) -> StreamExecutorResult<()> { let (ops, columns, visibility) = chunk.into_inner(); @@ -181,12 +181,17 @@ impl SimpleAggExecutor { let states = states.as_mut().unwrap(); // 2. Mark the state as dirty by filling prev states - states.may_mark_as_dirty(epoch).await?; + states.may_mark_as_dirty(epoch, state_tables).await?; // 3. Apply batch to each of the state (per agg_call) - for (agg_state, data) in states.managed_states.iter_mut().zip_eq(all_agg_data.iter()) { + for ((agg_state, data), state_table) in states + .managed_states + .iter_mut() + .zip_eq(all_agg_data.iter()) + .zip_eq(state_tables.iter_mut()) + { agg_state - .apply_batch(&ops, visibility.as_ref(), data, epoch) + .apply_batch(&ops, visibility.as_ref(), data, epoch, state_table) .await?; } @@ -234,7 +239,7 @@ impl SimpleAggExecutor { // --- Retrieve modified states and put the changes into the builders --- states - .build_changes(&mut builders, &mut new_ops, epoch) + .build_changes(&mut builders, &mut new_ops, epoch, state_tables) .await?; let columns: Vec = builders @@ -279,7 +284,7 @@ impl SimpleAggExecutor { &keyspace, chunk, epoch, - &state_tables, + &mut state_tables, ) .await?; } diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index d570ede85d439..0872f64a35450 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -30,6 +30,7 @@ use risingwave_common::hash::{HashCode, HashKey}; use risingwave_common::util::hash_util::CRC32FastBuilder; use risingwave_storage::table::state_table::StateTable; use risingwave_storage::{Keyspace, StateStore}; +use tokio::sync::RwLock; use super::{ expect_first_barrier, pk_input_arrays, Executor, PkDataTypes, PkIndicesRef, @@ -86,7 +87,7 @@ struct HashAggExecutorExtra { /// all of the aggregation functions in this executor should depend on same group of keys key_indices: Vec, - state_tables: Vec>, + state_tables: Arc>>>, } impl Executor for HashAggExecutor { @@ -125,7 +126,7 @@ impl HashAggExecutor { ks.clone(), agg_call, &key_indices, - &pk_indices, + &input_info.pk_indices, &schema, input.as_ref(), )); @@ -142,7 +143,7 @@ impl HashAggExecutor { keyspace, agg_calls, key_indices, - state_tables, + state_tables: Arc::new(RwLock::new(state_tables)), }, _phantom: PhantomData, }) @@ -200,16 +201,16 @@ impl HashAggExecutor { } async fn apply_chunk( - &HashAggExecutorExtra:: { + &mut HashAggExecutorExtra:: { ref key_indices, ref agg_calls, ref input_pk_indices, ref input_schema, ref keyspace, ref schema, - ref state_tables, + ref mut state_tables, .. - }: &HashAggExecutorExtra, + }: &mut HashAggExecutorExtra, state_map: &mut EvictableHashMap>>>, chunk: StreamChunk, epoch: u64, @@ -279,7 +280,7 @@ impl HashAggExecutor { input_pk_data_types.clone(), epoch, Some(hash_code), - state_tables, + &*state_tables.read().await, ) .await?, ), @@ -287,15 +288,21 @@ impl HashAggExecutor { }; // 2. Mark the state as dirty by filling prev states - states.may_mark_as_dirty(epoch).await?; + states + .may_mark_as_dirty(epoch, &*state_tables.read().await) + .await?; + let mut state_tables = state_tables.write().await; // 3. Apply batch to each of the state (per agg_call) - for (agg_state, data) in - states.managed_states.iter_mut().zip_eq(all_agg_data.iter()) + for ((agg_state, data), state_table) in states + .managed_states + .iter_mut() + .zip_eq(all_agg_data.iter()) + .zip_eq(state_tables.iter_mut()) { let data = data.iter().map(|d| &**d).collect_vec(); agg_state - .apply_batch(&ops, Some(&vis_map), &data, epoch) + .apply_batch(&ops, Some(&vis_map), &data, epoch, state_table) .await?; } @@ -329,10 +336,10 @@ impl HashAggExecutor { // --- Flush states to the state store --- // Some state will have the correct output only after their internal states have been // fully flushed. - let (write_batch, dirty_cnt) = { + let dirty_cnt = { let mut write_batch = store.start_write_batch(); let mut dirty_cnt = 0; - + let mut state_tables = state_tables.write().await; for states in state_map.values_mut() { if states.as_ref().unwrap().is_dirty() { dirty_cnt += 1; @@ -348,21 +355,19 @@ impl HashAggExecutor { } } - // Batch commit state table. - for state_table in state_tables.iter_mut() { - state_table.commit(epoch).await?; - } - - (write_batch, dirty_cnt) + dirty_cnt }; if dirty_cnt == 0 { // Nothing to flush. - assert!(write_batch.is_empty()); return Ok(()); } else { - write_batch.ingest(epoch).await?; + // Batch commit data. + for state_table in state_tables.write().await.iter_mut() { + state_table.commit(epoch).await?; + } + let state_tables = state_tables.read().await; // --- Produce the stream chunk --- let mut batches = IterChunks::chunks(state_map.iter_mut(), PROCESSING_WINDOW_SIZE); while let Some(batch) = batches.next() { @@ -379,7 +384,12 @@ impl HashAggExecutor { let appended = states .as_mut() .unwrap() - .build_changes(&mut builders[key_indices.len()..], &mut new_ops, epoch) + .build_changes( + &mut builders[key_indices.len()..], + &mut new_ops, + epoch, + &state_tables, + ) .await?; for _ in 0..appended { @@ -431,7 +441,7 @@ impl HashAggExecutor { let msg = msg?; match msg { Message::Chunk(chunk) => { - Self::apply_chunk(&extra, &mut state_map, chunk, epoch).await?; + Self::apply_chunk(&mut extra, &mut state_map, chunk, epoch).await?; } Message::Barrier(barrier) => { let next_epoch = barrier.epoch.curr; diff --git a/src/stream/src/executor/managed_state/aggregation/extreme.rs b/src/stream/src/executor/managed_state/aggregation/extreme.rs index 6768af6f6a7a3..353b5a038f679 100644 --- a/src/stream/src/executor/managed_state/aggregation/extreme.rs +++ b/src/stream/src/executor/managed_state/aggregation/extreme.rs @@ -12,24 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::marker::PhantomData; + use async_trait::async_trait; +use futures::{pin_mut, StreamExt}; use itertools::Itertools; use madsim::collections::BTreeMap; use risingwave_common::array::stream_chunk::{Op, Ops}; -use risingwave_common::array::{Array, ArrayImpl}; +use risingwave_common::array::{Array, ArrayImpl, Row}; use risingwave_common::buffer::Bitmap; -use risingwave_common::hash::{HashCode, VirtualNode}; +use risingwave_common::hash::HashCode; use risingwave_common::types::*; -use risingwave_common::util::value_encoding::{deserialize_cell, serialize_cell}; use risingwave_expr::expr::AggKind; -use risingwave_storage::storage_value::{StorageValue, ValueMeta}; +use risingwave_storage::table::state_table::StateTable; use risingwave_storage::write_batch::WriteBatch; use risingwave_storage::{Keyspace, StateStore}; -use super::super::flush_status::BtreeMapFlushStatus as FlushStatus; use super::extreme_serializer::{variants, ExtremePk, ExtremeSerializer}; use crate::executor::aggregation::{AggArgs, AggCall}; -use crate::executor::error::{StreamExecutorError, StreamExecutorResult}; +use crate::executor::error::StreamExecutorResult; use crate::executor::PkDataTypes; pub type ManagedMinState = GenericExtremeState; @@ -65,28 +66,22 @@ where /// it's top n. top_n: BTreeMap<(Option, ExtremePk), Datum>, - /// The actions that will be taken on next flush - flush_buffer: BTreeMap<(Option, ExtremePk), FlushStatus>, - /// Number of items in the state including those not in top n cache but in state store. total_count: usize, /// Number of entries to retain in memory after each flush. top_n_count: Option, - /// Data type of the sort column - data_type: DataType, - - /// The keyspace to operate on. - keyspace: Keyspace, + // TODO: Remove this phantom to get rid of S: StateStore. + _phantom_data: PhantomData, /// The sort key serializer + /// TODO: Remove this soon. serializer: ExtremeSerializer, - /// Computed via consistent hash. The value is to be set in value meta and used for grouping - /// the kv together in storage. Each extreme state will have the same value of virtual node, - /// since it is computed on group key. - vnode: VirtualNode, + /// Primary key to look up in relational table. For value state, there is only one row. + /// If None, the pk is empty vector (simple agg). If not None, the pk is group key (hash agg). + group_key: Option, } /// A trait over all table-structured states. @@ -103,16 +98,25 @@ pub trait ManagedTableState: Send + Sync + 'static { visibility: Option<&Bitmap>, data: &[&ArrayImpl], epoch: u64, + state_table: &mut StateTable, ) -> StreamExecutorResult<()>; /// Get the output of the state. Must flush before getting output. - async fn get_output(&mut self, epoch: u64) -> StreamExecutorResult; + async fn get_output( + &mut self, + epoch: u64, + state_table: &StateTable, + ) -> StreamExecutorResult; /// Check if this state needs a flush. fn is_dirty(&self) -> bool; /// Flush the internal state to a write batch. - fn flush(&mut self, write_batch: &mut WriteBatch) -> StreamExecutorResult<()>; + fn flush( + &mut self, + write_batch: &mut WriteBatch, + state_table: &mut StateTable, + ) -> StreamExecutorResult<()>; } impl GenericExtremeState @@ -124,23 +128,22 @@ where /// always be retained when flushing the managed state. Otherwise, we will only retain n entries /// after each flush. pub async fn new( - keyspace: Keyspace, + _keyspace: Keyspace, data_type: DataType, top_n_count: Option, row_count: usize, pk_data_types: PkDataTypes, - group_key_hash_code: HashCode, + _group_key_hash_code: HashCode, + group_key: Option<&Row>, ) -> StreamExecutorResult { // Create the internal state based on the value we get. Ok(Self { top_n: BTreeMap::new(), - flush_buffer: BTreeMap::new(), total_count: row_count, - keyspace, + _phantom_data: PhantomData::default(), top_n_count, - data_type: data_type.clone(), serializer: ExtremeSerializer::new(data_type, pk_data_types), - vnode: group_key_hash_code.to_vnode(), + group_key: group_key.cloned(), }) } @@ -177,6 +180,7 @@ where ops: Ops<'_>, visibility: Option<&Bitmap>, data: &[&ArrayImpl], + state_table: &mut StateTable, ) -> StreamExecutorResult<()> { debug_assert!(super::verify_batch(ops, visibility, data)); @@ -205,15 +209,23 @@ where } // sort key may be null - let key = option_to_owned_scalar(&key); + let key: Option = option_to_owned_scalar(&key); // Concat pk with the original key to create a composed key let composed_key = ( key.clone(), // Collect pk from columns - pk_columns.iter().map(|col| col.datum_at(id)).collect(), + pk_columns + .iter() + .map(|col| col.datum_at(id)) + .collect::(), ); + // Get relational pk and value. + let sort_key = key.map(|key| key.into()); + let relational_value = + self.get_relational_value(sort_key.clone(), composed_key.1.clone()); + match op { Op::Insert | Op::UpdateInsert => { let mut do_insert = false; @@ -239,19 +251,15 @@ where _ => unreachable!(), } } - - let value: Option = key.clone().map(|x| x.into()); - if do_insert { - self.top_n.insert(composed_key.clone(), value.clone()); + self.top_n.insert(composed_key.clone(), sort_key); } - - FlushStatus::do_insert(self.flush_buffer.entry(composed_key), value); + state_table.insert(relational_value)?; self.total_count += 1; } Op::Delete | Op::UpdateDelete => { self.top_n.remove(&composed_key); - FlushStatus::do_delete(self.flush_buffer.entry(composed_key)); + state_table.delete(relational_value)?; self.total_count -= 1; } } @@ -279,14 +287,17 @@ where None } - async fn get_output_inner(&mut self, epoch: u64) -> StreamExecutorResult { + async fn get_output_inner( + &mut self, + epoch: u64, + state_table: &StateTable, + ) -> StreamExecutorResult { // To make things easier, we do not allow get_output before flushing. Otherwise we will need // to merge data from flush_buffer and state store, which is hard to implement. // // To make ExtremeState produce the correct result, the write batch must be flushed into the // state store before getting the output. Note that calling `.flush()` is not enough, as it // only generates a write batch without flushing to store. - debug_assert!(!self.is_dirty()); // Firstly, check if datum is available in cache. if let Some(v) = self.get_output_from_cache() { @@ -297,15 +308,32 @@ where // To future developers: please make **SURE** you have taken `EXTREME_TYPE` into // account. EXTREME_MIN and EXTREME_MAX will significantly impact the // following logic. - let all_data = self.keyspace.scan(self.top_n_count, epoch).await?; - - for (raw_key, mut raw_value) in all_data { - // let mut deserializer = value_encoding::Deserializer::new(raw_value); - let value = deserialize_cell(&mut raw_value, &self.data_type) - .map_err(StreamExecutorError::serde_error)?; - let key = value.clone().map(|x| x.try_into().unwrap()); - let pks = self.serializer.get_pk(&raw_key[..])?; - self.top_n.insert((key, pks), value); + let all_data_iter = state_table + .iter_with_pk_prefix(self.group_key.as_ref().unwrap_or(&Row::default()), epoch) + .await?; + pin_mut!(all_data_iter); + + for _ in 0..self.top_n_count.unwrap_or(usize::MAX) { + if let Some(inner) = all_data_iter.next().await { + let row = inner?; + + // Get the agg call value. + let value = row[row.0.len() - 1].clone(); + + // Get sort key and extreme pk. + let sort_key = value.as_ref().map(|v| v.clone().try_into().unwrap()); + let mut extreme_pk = ExtremePk::with_capacity(1); + let group_key_len = self.group_key.as_ref().map_or(0, |row| row.size()); + // The layout is group_key/sort_key/extreme_pk/agg_call value. So the range + // should be [group_key_len + 1, row.0.len() - 1). + for extreme_pk_index in group_key_len + 1..row.0.len() - 1 { + extreme_pk.push(row[extreme_pk_index].clone()); + } + + self.top_n.insert((sort_key, extreme_pk), value); + } else { + break; + } } if let Some(v) = self.get_output_from_cache() { @@ -317,42 +345,25 @@ where } /// Flush the internal state to a write batch. - fn flush_inner(&mut self, write_batch: &mut WriteBatch) -> StreamExecutorResult<()> { - // Generally, we require the state the be dirty before flushing. However, it is possible - // that after a sequence of operations, the flush buffer becomes empty. Then, the - // state becomes "dirty", but we do not need to flush anything. - if !self.is_dirty() { - self.retain_top_n(); - return Ok(()); - } - - let mut local = write_batch.prefixify(&self.keyspace); - let value_meta = ValueMeta::with_vnode(self.vnode); - - // TODO: we can populate the cache while flushing, but that's hard. - - for ((key, pks), v) in std::mem::take(&mut self.flush_buffer) { - let key_encoded = self.serializer.serialize(key, &pks)?; - match v.into_option() { - Some(v) => { - local.put( - key_encoded, - StorageValue::new_put( - value_meta, - serialize_cell(&v).map_err(StreamExecutorError::serde_error)?, - ), - ); - } - None => { - local.delete_with_value_meta(key_encoded, value_meta); - } - } - } - + /// TODO: Revisit all `.flush` and remove if necessary. + fn flush_inner(&mut self) -> StreamExecutorResult<()> { self.retain_top_n(); - Ok(()) } + + /// Assemble value for relational table used by extreme state. Should be `group_key` + + /// `sort_key` + input pk + agg call value. + fn get_relational_value(&self, sort_key: Datum, extreme_pk: ExtremePk) -> Row { + let mut sort_key_vec = if let Some(group_key) = self.group_key.as_ref() { + group_key.0.to_vec() + } else { + vec![] + }; + sort_key_vec.push(sort_key.clone()); + sort_key_vec.extend(extreme_pk.into_iter()); + sort_key_vec.push(sort_key); + Row::new(sort_key_vec) + } } #[async_trait] @@ -368,21 +379,32 @@ where visibility: Option<&Bitmap>, data: &[&ArrayImpl], _epoch: u64, + state_table: &mut StateTable, ) -> StreamExecutorResult<()> { - self.apply_batch_inner(ops, visibility, data).await + self.apply_batch_inner(ops, visibility, data, state_table) + .await } - async fn get_output(&mut self, epoch: u64) -> StreamExecutorResult { - self.get_output_inner(epoch).await + async fn get_output( + &mut self, + epoch: u64, + state_table: &StateTable, + ) -> StreamExecutorResult { + self.get_output_inner(epoch, state_table).await } /// Check if this state needs a flush. + /// TODO: Remove this. fn is_dirty(&self) -> bool { - !self.flush_buffer.is_empty() + unreachable!("Should not call this function anymore, check state table for dirty data"); } - fn flush(&mut self, write_batch: &mut WriteBatch) -> StreamExecutorResult<()> { - self.flush_inner(write_batch) + fn flush( + &mut self, + _write_batch: &mut WriteBatch, + _state_table: &mut StateTable, + ) -> StreamExecutorResult<()> { + self.flush_inner() } } @@ -393,6 +415,7 @@ pub async fn create_streaming_extreme_state( top_n_count: Option, pk_data_types: PkDataTypes, key_hash_code: Option, + pk: Option<&Row>, ) -> StreamExecutorResult>> { match &agg_call.args { AggArgs::Unary(x, _) => { @@ -420,7 +443,8 @@ pub async fn create_streaming_extreme_state( top_n_count, row_count, pk_data_types, - key_hash_code.unwrap_or_default() + key_hash_code.unwrap_or_default(), + pk ).await?, )), (AggKind::Min, $( $kind )|+) => Ok(Box::new( @@ -430,7 +454,8 @@ pub async fn create_streaming_extreme_state( top_n_count, row_count, pk_data_types, - key_hash_code.unwrap_or_default() + key_hash_code.unwrap_or_default(), + pk ).await?, )), )* @@ -463,8 +488,9 @@ mod tests { use madsim::collections::{BTreeSet, HashSet}; use madsim::rand::prelude::*; use risingwave_common::array::{I64Array, Op}; - use risingwave_common::catalog::TableId; + use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId}; use risingwave_common::types::ScalarImpl; + use risingwave_common::util::sort_util::OrderType; use risingwave_storage::memory::MemoryStateStore; use smallvec::smallvec; @@ -474,6 +500,8 @@ mod tests { async fn test_managed_extreme_state() { let store = MemoryStateStore::new(); let keyspace = Keyspace::table_root(store.clone(), &TableId::from(0x2333)); + let mut state_table = state_table_create_helper(keyspace.clone(), 2, OrderType::Ascending); + let mut managed_state = ManagedMinState::<_, I64Array>::new( keyspace.clone(), DataType::Int64, @@ -481,10 +509,11 @@ mod tests { 0, PkDataTypes::new(), HashCode(567), + None, ) .await .unwrap(); - assert!(!managed_state.is_dirty()); + assert!(!state_table.is_dirty()); let mut epoch: u64 = 0; // insert 0, 10, 20 @@ -496,19 +525,23 @@ mod tests { .unwrap() .into()], epoch, + &mut state_table, ) .await .unwrap(); - assert!(managed_state.is_dirty()); + assert!(state_table.is_dirty()); // flush to write batch and write to state store let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); + state_table.commit(epoch).await.unwrap(); // The minimum should be 0 assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Int64(0)) ); @@ -524,6 +557,7 @@ mod tests { .into(), ], epoch, + &mut state_table, ) .await .unwrap(); @@ -531,12 +565,15 @@ mod tests { // flush to write batch and write to state store epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); + state_table.commit(epoch).await.unwrap(); // The minimum should be 0 assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Int64(0)) ); @@ -551,6 +588,7 @@ mod tests { .into(), ], epoch, + &mut state_table, ) .await .unwrap(); @@ -558,12 +596,15 @@ mod tests { // flush to write batch and write to state store epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); + state_table.commit(epoch).await.unwrap(); // The minimum should be 20 assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Int64(20)) ); @@ -574,6 +615,7 @@ mod tests { None, &[&I64Array::from_slice(&[Some(20), Some(27)]).unwrap().into()], epoch, + &mut state_table, ) .await .unwrap(); @@ -581,12 +623,15 @@ mod tests { // flush to write batch and write to state store epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); + state_table.commit(epoch).await.unwrap(); // The minimum should be 25 assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Int64(25)) ); @@ -597,6 +642,7 @@ mod tests { None, &[&I64Array::from_slice(&[Some(25)]).unwrap().into()], epoch, + &mut state_table, ) .await .unwrap(); @@ -604,12 +650,15 @@ mod tests { // flush to write batch and write to state store epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); + state_table.commit(epoch).await.unwrap(); // The minimum should be 30 assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Int64(30)) ); @@ -624,13 +673,14 @@ mod tests { row_count, PkDataTypes::new(), HashCode(567), + None, ) .await .unwrap(); // The minimum should still be 30 assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Int64(30)) ); } @@ -655,9 +705,37 @@ mod tests { test_replicated_value_with_null::<{ variants::EXTREME_MAX }>().await } + fn state_table_create_helper( + keyspace: Keyspace, + column_cnt: usize, + order_type: OrderType, + ) -> StateTable { + let mut column_descs = Vec::with_capacity(column_cnt); + for id in 0..column_cnt { + column_descs.push(ColumnDesc::unnamed( + ColumnId::new(id as i32), + DataType::Int64, + )); + } + let relational_pk_len = column_descs.len() - 1; + StateTable::new( + keyspace, + column_descs, + vec![order_type; relational_pk_len], + None, + (0..relational_pk_len).collect(), + ) + } + async fn test_replicated_value_not_null() { let store = MemoryStateStore::new(); let keyspace = Keyspace::table_root(store.clone(), &TableId::from(0x2333)); + let order_type = if EXTREME_TYPE == variants::EXTREME_MAX { + OrderType::Descending + } else { + OrderType::Ascending + }; + let mut state_table = state_table_create_helper(keyspace.clone(), 3, order_type); let mut managed_state = GenericExtremeState::<_, I64Array, EXTREME_TYPE>::new( keyspace.clone(), @@ -666,10 +744,11 @@ mod tests { 0, smallvec![DataType::Int64], HashCode(567), + None, ) .await .unwrap(); - assert!(!managed_state.is_dirty()); + assert!(!state_table.is_dirty()); let value_buffer = I64Array::from_slice(&[Some(1), Some(1), Some(4), Some(5), Some(1), Some(4)]) @@ -707,43 +786,72 @@ mod tests { .into(), ], epoch, + &mut state_table, ) .await .unwrap(); // insert 1 1 4 5 1 4 managed_state - .apply_batch(&[Op::Insert; 6], None, &[&value_buffer, &pk_buffer], epoch) + .apply_batch( + &[Op::Insert; 6], + None, + &[&value_buffer, &pk_buffer], + epoch, + &mut state_table, + ) .await .unwrap(); // flush let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); // The minimum should be 1, or the maximum should be 5 - assert_eq!(managed_state.get_output(epoch).await.unwrap(), extreme); + assert_eq!( + managed_state.get_output(epoch, &state_table).await.unwrap(), + extreme + ); // delete 1 1 4 5 1 4 managed_state - .apply_batch(&[Op::Delete; 6], None, &[&value_buffer, &pk_buffer], epoch) + .apply_batch( + &[Op::Delete; 6], + None, + &[&value_buffer, &pk_buffer], + epoch, + &mut state_table, + ) .await .unwrap(); // flush epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); // The minimum should still be 1, or the maximum should still be 5 - assert_eq!(managed_state.get_output(epoch).await.unwrap(), extreme); + assert_eq!( + managed_state.get_output(epoch, &state_table).await.unwrap(), + extreme + ); } async fn test_replicated_value_with_null() { let store = MemoryStateStore::new(); let keyspace = Keyspace::table_root(store.clone(), &TableId::from(0x2333)); + let order_type = if EXTREME_TYPE == variants::EXTREME_MAX { + OrderType::Descending + } else { + OrderType::Ascending + }; + let mut state_table = state_table_create_helper(keyspace.clone(), 3, order_type); let mut managed_state = GenericExtremeState::<_, I64Array, EXTREME_TYPE>::new( keyspace.clone(), @@ -752,10 +860,11 @@ mod tests { 0, smallvec![DataType::Int64], HashCode(567), + None, ) .await .unwrap(); - assert!(!managed_state.is_dirty()); + assert!(!state_table.is_dirty()); let value_buffer = I64Array::from_slice(&[Some(1), Some(1), Some(4), Some(5), Some(1), Some(4)]) @@ -793,6 +902,7 @@ mod tests { .into(), ], epoch, + &mut state_table, ) .await .unwrap(); @@ -809,38 +919,61 @@ mod tests { None, &[null_buffer, null_pk_buffer], epoch, + &mut state_table, ) .await .unwrap(); // insert 1 1 4 5 1 4 managed_state - .apply_batch(&[Op::Insert; 6], None, &[&value_buffer, &pk_buffer], epoch) + .apply_batch( + &[Op::Insert; 6], + None, + &[&value_buffer, &pk_buffer], + epoch, + &mut state_table, + ) .await .unwrap(); // flush let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); // The minimum should be None, or the maximum should be 5 - assert_eq!(managed_state.get_output(epoch).await.unwrap(), extreme); + assert_eq!( + managed_state.get_output(epoch, &state_table).await.unwrap(), + extreme + ); // delete 1 1 4 5 1 4 managed_state - .apply_batch(&[Op::Delete; 6], None, &[&value_buffer, &pk_buffer], epoch) + .apply_batch( + &[Op::Delete; 6], + None, + &[&value_buffer, &pk_buffer], + epoch, + &mut state_table, + ) .await .unwrap(); // flush epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); // The minimum should still be None, or the maximum should still be 5 - assert_eq!(managed_state.get_output(epoch).await.unwrap(), extreme); + assert_eq!( + managed_state.get_output(epoch, &state_table).await.unwrap(), + extreme + ); } #[tokio::test] @@ -854,10 +987,13 @@ mod tests { 0, PkDataTypes::new(), HashCode(567), + None, ) .await .unwrap(); - assert!(!managed_state.is_dirty()); + let mut state_table = state_table_create_helper(keyspace.clone(), 2, OrderType::Ascending); + + assert!(!state_table.is_dirty()); let value_buffer = I64Array::from_slice(&[Some(0), Some(1), Some(2), Some(3), Some(4), Some(5)]) @@ -871,13 +1007,20 @@ mod tests { None, &[&I64Array::from_slice(&[Some(6)]).unwrap().into()], epoch, + &mut state_table, ) .await .unwrap(); for i in 0..100 { managed_state - .apply_batch(&[Op::Insert; 6], None, &[&value_buffer], epoch) + .apply_batch( + &[Op::Insert; 6], + None, + &[&value_buffer], + epoch, + &mut state_table, + ) .await .unwrap(); @@ -886,24 +1029,34 @@ mod tests { // flush to write batch and write to state store epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); } managed_state - .apply_batch(&[Op::Delete; 6], None, &[&value_buffer], epoch) + .apply_batch( + &[Op::Delete; 6], + None, + &[&value_buffer], + epoch, + &mut state_table, + ) .await .unwrap(); // flush to write batch and write to state store epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); // The minimum should be 6 assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Int64(6)) ); } @@ -927,6 +1080,20 @@ mod tests { let store = MemoryStateStore::new(); let keyspace = Keyspace::table_root(store.clone(), &TableId::from(0x2333)); + // let mut state_table = StateTable::new( + // keyspace.clone(), + // vec![ColumnDesc::unnamed(ColumnId::new(0), DataType::Int64)], + // vec![], + // None, + // vec![], + // ); + let order_type = if EXTREME_TYPE == variants::EXTREME_MAX { + OrderType::Descending + } else { + OrderType::Ascending + }; + let mut state_table = state_table_create_helper(keyspace.clone(), 2, order_type); + let mut managed_state = GenericExtremeState::<_, I64Array, EXTREME_TYPE>::new( keyspace.clone(), DataType::Int64, @@ -934,6 +1101,7 @@ mod tests { 0, PkDataTypes::new(), HashCode(567), + None, ) .await .unwrap(); @@ -954,7 +1122,7 @@ mod tests { let epoch: u64 = 0; managed_state - .apply_batch(&ops, None, &[&arr.into()], epoch) + .apply_batch(&ops, None, &[&arr.into()], epoch, &mut state_table) .await .unwrap(); @@ -983,17 +1151,20 @@ mod tests { let arr = I64Array::from_slice(&to_be_delete.iter().map(|x| Some(*x)).collect_vec()).unwrap(); managed_state - .apply_batch(&ops, None, &[&arr.into()], epoch) + .apply_batch(&ops, None, &[&arr.into()], epoch, &mut state_table) .await .unwrap(); // flush to write batch and write to state store let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); + state_table.commit(epoch).await.unwrap(); let value = managed_state - .get_output(epoch) + .get_output(epoch, &state_table) .await .unwrap() .map(|x| x.into_int64()); @@ -1010,9 +1181,10 @@ mod tests { managed_state: &mut impl ManagedTableState, keyspace: &Keyspace, epoch: u64, + state_table: &mut StateTable, ) { let mut write_batch = keyspace.state_store().start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state.flush(&mut write_batch, state_table).unwrap(); write_batch.ingest(epoch).await.unwrap(); } @@ -1032,10 +1204,13 @@ mod tests { 0, PkDataTypes::new(), HashCode(567), + None, ) .await .unwrap(); - assert!(!managed_state.is_dirty()); + let mut state_table = state_table_create_helper(keyspace.clone(), 2, OrderType::Ascending); + + assert!(!state_table.is_dirty()); let value_buffer = I64Array::from_slice(&[Some(1), Some(2), Some(3), Some(4), Some(5), Some(7)]) @@ -1044,7 +1219,13 @@ mod tests { let epoch: u64 = 0; managed_state - .apply_batch(&[Op::Insert; 6], None, &[&value_buffer], epoch) + .apply_batch( + &[Op::Insert; 6], + None, + &[&value_buffer], + epoch, + &mut state_table, + ) .await .unwrap(); @@ -1054,12 +1235,13 @@ mod tests { None, &[&I64Array::from_slice(&[Some(6)]).unwrap().into()], epoch, + &mut state_table, ) .await .unwrap(); // Now we have 1 to 7 in the state store. - helper_flush(&mut managed_state, &keyspace, epoch).await; + helper_flush(&mut managed_state, &keyspace, epoch, &mut state_table).await; // Delete 6, insert 6, delete 6 managed_state @@ -1070,12 +1252,13 @@ mod tests { .unwrap() .into()], epoch, + &mut state_table, ) .await .unwrap(); // 6 should be deleted by now - helper_flush(&mut managed_state, &keyspace, epoch).await; + helper_flush(&mut managed_state, &keyspace, epoch, &mut state_table).await; let value_buffer = I64Array::from_slice(&[Some(1), Some(2), Some(3), Some(4), Some(5)]) .unwrap() @@ -1083,14 +1266,20 @@ mod tests { // delete all remaining items managed_state - .apply_batch(&[Op::Delete; 5], None, &[&value_buffer], epoch) + .apply_batch( + &[Op::Delete; 5], + None, + &[&value_buffer], + epoch, + &mut state_table, + ) .await .unwrap(); - helper_flush(&mut managed_state, &keyspace, epoch).await; + helper_flush(&mut managed_state, &keyspace, epoch, &mut state_table).await; assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Int64(7)) ); } diff --git a/src/stream/src/executor/managed_state/aggregation/extreme_serializer.rs b/src/stream/src/executor/managed_state/aggregation/extreme_serializer.rs index 1e60e921e7e16..45121bedb252b 100644 --- a/src/stream/src/executor/managed_state/aggregation/extreme_serializer.rs +++ b/src/stream/src/executor/managed_state/aggregation/extreme_serializer.rs @@ -51,6 +51,7 @@ impl ExtremeSerializer { } } + #[allow(dead_code)] fn is_reversed_order(&self) -> bool { match EXTREME_TYPE { variants::EXTREME_MAX => true, @@ -62,6 +63,7 @@ impl ExtremeSerializer { /// Serialize key and `pk` (or, `row_id`s) into a sort key /// /// TODO: support `&K` instead of `K` as parameter. + #[allow(dead_code)] pub fn serialize(&self, key: Option, pk: &ExtremePk) -> StreamExecutorResult> { let mut serializer = memcomparable::Serializer::new(vec![]); serializer.set_reverse(self.is_reversed_order()); @@ -82,6 +84,7 @@ impl ExtremeSerializer { } /// Extract the pks from the sort key + #[allow(dead_code)] pub fn get_pk(&self, data: &[u8]) -> StreamExecutorResult { if self.pk_data_types.is_empty() { return Ok(ExtremePk::default()); diff --git a/src/stream/src/executor/managed_state/aggregation/mod.rs b/src/stream/src/executor/managed_state/aggregation/mod.rs index d00d429852adc..735745b8ae506 100644 --- a/src/stream/src/executor/managed_state/aggregation/mod.rs +++ b/src/stream/src/executor/managed_state/aggregation/mod.rs @@ -68,18 +68,27 @@ impl ManagedStateImpl { visibility: Option<&Bitmap>, data: &[&ArrayImpl], epoch: u64, + state_table: &mut StateTable, ) -> StreamExecutorResult<()> { match self { Self::Value(state) => state.apply_batch(ops, visibility, data).await, - Self::Table(state) => state.apply_batch(ops, visibility, data, epoch).await, + Self::Table(state) => { + state + .apply_batch(ops, visibility, data, epoch, state_table) + .await + } } } /// Get the output of the state. Must flush before getting output. - pub async fn get_output(&mut self, epoch: u64) -> StreamExecutorResult { + pub async fn get_output( + &mut self, + epoch: u64, + state_table: &StateTable, + ) -> StreamExecutorResult { match self { Self::Value(state) => state.get_output().await, - Self::Table(state) => state.get_output(epoch).await, + Self::Table(state) => state.get_output(epoch, state_table).await, } } @@ -99,7 +108,7 @@ impl ManagedStateImpl { ) -> StreamExecutorResult<()> { match self { Self::Value(state) => state.flush(write_batch, state_table).await, - Self::Table(state) => state.flush(write_batch), + Self::Table(state) => state.flush(write_batch, state_table), } } @@ -137,6 +146,7 @@ impl ManagedStateImpl { Some(1024), pk_data_types, key_hash_code, + pk, ) .await?, )) diff --git a/src/stream/src/executor/managed_state/aggregation/string_agg.rs b/src/stream/src/executor/managed_state/aggregation/string_agg.rs index b9df0dc3fcca1..f03adb706920d 100644 --- a/src/stream/src/executor/managed_state/aggregation/string_agg.rs +++ b/src/stream/src/executor/managed_state/aggregation/string_agg.rs @@ -23,6 +23,7 @@ use risingwave_common::types::{DataType, Datum, ScalarImpl}; use risingwave_common::util::ordered::OrderedArraysSerializer; use risingwave_common::util::value_encoding::{deserialize_cell, serialize_cell}; use risingwave_storage::storage_value::StorageValue; +use risingwave_storage::table::state_table::StateTable; use risingwave_storage::write_batch::WriteBatch; use risingwave_storage::{Keyspace, StateStore}; @@ -161,6 +162,7 @@ impl ManagedTableState for ManagedStringAggState { visibility: Option<&Bitmap>, data: &[&ArrayImpl], epoch: u64, + _state_table: &mut StateTable, ) -> StreamExecutorResult<()> { debug_assert!(super::verify_batch(ops, visibility, data)); for sort_key_index in &self.sort_key_indices { @@ -207,7 +209,11 @@ impl ManagedTableState for ManagedStringAggState { Ok(()) } - async fn get_output(&mut self, epoch: u64) -> StreamExecutorResult { + async fn get_output( + &mut self, + epoch: u64, + _state_table: &StateTable, + ) -> StreamExecutorResult { // We allow people to get output when the data is dirty. // As this is easier compared to `ManagedMinState` as we have a all-or-nothing cache policy // here. @@ -241,7 +247,11 @@ impl ManagedTableState for ManagedStringAggState { self.dirty } - fn flush(&mut self, write_batch: &mut WriteBatch) -> StreamExecutorResult<()> { + fn flush( + &mut self, + write_batch: &mut WriteBatch, + _state_table: &mut StateTable, + ) -> StreamExecutorResult<()> { if !self.is_dirty() { return Ok(()); } @@ -308,6 +318,10 @@ mod tests { .unwrap() } + fn mock_state_table(keyspace: Keyspace) -> StateTable { + StateTable::new(keyspace, vec![], vec![], None, vec![]) + } + #[tokio::test] async fn test_managed_string_agg_state() { let keyspace = create_in_memory_keyspace(); @@ -315,6 +329,7 @@ mod tests { let mut managed_state = create_managed_state(&keyspace, 0).await; assert!(!managed_state.is_dirty()); let mut epoch: u64 = 0; + let mut state_table = mock_state_table(keyspace.clone()); // Insert. managed_state @@ -330,6 +345,7 @@ mod tests { .into(), ], epoch, + &mut state_table, ) .await .unwrap(); @@ -337,12 +353,14 @@ mod tests { // Check output after insertion. assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Utf8("ghi||def||abc".to_string())) ); let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); assert!(!managed_state.is_dirty()); @@ -360,6 +378,7 @@ mod tests { .into(), ], epoch, + &mut state_table, ) .await .unwrap(); @@ -367,13 +386,15 @@ mod tests { // Check output after insertion and deletion. assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Utf8("ghi||def||def||abc".to_string())) ); epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); assert!(!managed_state.is_dirty()); @@ -391,6 +412,7 @@ mod tests { .into(), ], epoch, + &mut state_table, ) .await .unwrap(); @@ -399,19 +421,21 @@ mod tests { // Check output after deletion. assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Utf8("ghi".to_string())) ); epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); assert!(!managed_state.is_dirty()); // Check output after flush. assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Utf8("ghi".to_string())) ); @@ -424,7 +448,7 @@ mod tests { assert!(!managed_state.is_dirty()); // Get the output after recovery assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Utf8("ghi".to_string())) ); @@ -442,12 +466,13 @@ mod tests { .into(), ], epoch, + &mut state_table, ) .await .unwrap(); assert!(managed_state.is_dirty()); assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Utf8("ghi||ghi".to_string())) ); // Check dirtiness after getting the output. @@ -466,11 +491,15 @@ mod tests { &I64Array::from_slice(&[Some(5), Some(6)]).unwrap().into(), ], epoch, + &mut state_table, ) .await .unwrap(); assert!(managed_state.is_dirty()); - assert_eq!(managed_state.get_output(epoch).await.unwrap(), None,); + assert_eq!( + managed_state.get_output(epoch, &state_table).await.unwrap(), + None, + ); assert_eq!(managed_state.get_row_count(), 0); managed_state @@ -484,13 +513,16 @@ mod tests { &I64Array::from_slice(&[Some(7), Some(8)]).unwrap().into(), ], epoch, + &mut state_table, ) .await .unwrap(); epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); assert!(!managed_state.is_dirty()); let row_count = managed_state.get_row_count(); @@ -509,17 +541,20 @@ mod tests { &I64Array::from_slice(&[Some(7), Some(9)]).unwrap().into(), ], epoch, + &mut state_table, ) .await .unwrap(); assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Utf8("miko||miko".to_string())) ); epoch += 1; let mut write_batch = store.start_write_batch(); - managed_state.flush(&mut write_batch).unwrap(); + managed_state + .flush(&mut write_batch, &mut state_table) + .unwrap(); write_batch.ingest(epoch).await.unwrap(); assert!(!managed_state.is_dirty()); @@ -528,7 +563,7 @@ mod tests { drop(managed_state); let mut managed_state = create_managed_state(&keyspace, row_count).await; assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Utf8("miko||miko".to_string())) ); @@ -546,11 +581,12 @@ mod tests { .into(), ], epoch, + &mut state_table, ) .await .unwrap(); assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Utf8("simple||naive||miko".to_string())) ); @@ -561,7 +597,7 @@ mod tests { // As we didn't flush the changes, the result should be the same as the result before last // changes. assert_eq!( - managed_state.get_output(epoch).await.unwrap(), + managed_state.get_output(epoch, &state_table).await.unwrap(), Some(ScalarImpl::Utf8("miko||miko".to_string())) ); } diff --git a/src/stream/src/executor/managed_state/aggregation/value.rs b/src/stream/src/executor/managed_state/aggregation/value.rs index ce34e76229864..2a532b36cac4a 100644 --- a/src/stream/src/executor/managed_state/aggregation/value.rs +++ b/src/stream/src/executor/managed_state/aggregation/value.rs @@ -92,7 +92,7 @@ impl ManagedValueState { /// Get the output of the state. Note that in our case, getting the output is very easy, as the /// output is the same as the aggregation state. In other aggregators, like min and max, /// `get_output` might involve a scan from the state store. - pub async fn get_output(&mut self) -> StreamExecutorResult { + pub async fn get_output(&self) -> StreamExecutorResult { debug_assert!(!self.is_dirty()); self.state.get_output() } @@ -191,7 +191,7 @@ mod tests { ); // reload the state and check the output - let mut managed_state = + let managed_state = ManagedValueState::new(create_test_count_state(), None, None, &state_table) .await .unwrap(); @@ -262,7 +262,7 @@ mod tests { ); // reload the state and check the output - let mut managed_state = + let managed_state = ManagedValueState::new(create_test_max_agg_append_only(), None, None, &state_table) .await .unwrap(); diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index 90341a6aa1107..8b099fd78286e 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -167,8 +167,8 @@ pub trait ExprFn = Fn(&DataChunk) -> Result + Send + Sync + 'static; #[derive(Debug, PartialEq, Clone, Default)] pub struct AddOutput { - map: HashMap<(ActorId, DispatcherId), Vec>, - splits: HashMap>, + pub map: HashMap<(ActorId, DispatcherId), Vec>, + pub splits: HashMap>, } #[derive(Debug, Clone, PartialEq)] diff --git a/src/stream/src/executor/source.rs b/src/stream/src/executor/source.rs index e0023eeae9f5c..71b7393b9f544 100644 --- a/src/stream/src/executor/source.rs +++ b/src/stream/src/executor/source.rs @@ -27,8 +27,8 @@ use risingwave_connector::state::SourceStateHandler; use risingwave_connector::{ConnectorState, SplitImpl, SplitMetaData}; use risingwave_source::*; use risingwave_storage::{Keyspace, StateStore}; -use tokio::sync::mpsc::{channel, Receiver, UnboundedReceiver}; -use tokio::sync::{oneshot, Notify}; +use tokio::sync::mpsc::UnboundedReceiver; +use tokio::sync::{Mutex, Notify}; use tokio::time::Instant; use super::error::StreamExecutorError; @@ -134,7 +134,7 @@ impl SourceExecutor { struct SourceReader { /// The reader for stream source. - stream_reader: Box, + stream_reader: Arc>>, /// The reader for barrier. barrier_receiver: UnboundedReceiver, /// Expected barrier latency in ms. If there are no barrier within the expected barrier @@ -145,49 +145,34 @@ struct SourceReader { impl SourceReader { #[try_stream(ok = StreamChunkWithState, error = RwError)] async fn stream_reader( - mut stream_reader: Box, + stream_reader: Arc>>, notifier: Arc, expected_barrier_latency_ms: u64, - mut inject_source_rx: Receiver<(Box, oneshot::Sender<()>)>, ) { - let (msg_tx, mut msg_rx) = channel::>(1); - let handler = tokio::task::spawn(async move { - loop { - let now = Instant::now(); - - // We allow data to flow for `expected_barrier_latency_ms` milliseconds. - while now.elapsed().as_millis() < expected_barrier_latency_ms as u128 { - tokio::select! { - biased; - reader = inject_source_rx.recv() => { - if let Some((new_reader, tx)) = reader { - stream_reader = new_reader; - tx.send(()).unwrap(); - } - } - chunk = stream_reader.next() => { msg_tx.send(chunk).await.unwrap(); } + 'outer: loop { + let now = Instant::now(); + + // We allow data to flow for `expected_barrier_latency_ms` milliseconds. + while now.elapsed().as_millis() < expected_barrier_latency_ms as u128 { + let mut reader_guard = stream_reader.lock().await; + let chunk_result = reader_guard.next().await; + drop(reader_guard); + match chunk_result { + Ok(chunk) => yield chunk, + Err(e) => { + error!("hang up stream reader due to polling error: {}", e); + break 'outer; } - } + }; + } - // Here we consider two cases: - // - // 1. Barrier arrived before waiting for notified. In this case, this await will - // complete instantly, and we will continue to produce new data. - // 2. Barrier arrived after waiting for notified. Then source will be stalled. + // Here we consider two cases: + // + // 1. Barrier arrived before waiting for notified. In this case, this await will + // complete instantly, and we will continue to produce new data. + // 2. Barrier arrived after waiting for notified. Then source will be stalled. - notifier.notified().await; - } - }); - 'outer: loop { - match msg_rx.recv().await.unwrap() { - Ok(chunk) => yield chunk, - Err(e) => { - error!("hang up stream reader due to polling error: {}", e); - handler.abort(); - // Then hang up this stream by breaking the loop. - break 'outer; - } - } + notifier.notified().await; } futures::future::pending().await @@ -206,7 +191,6 @@ impl SourceReader { fn into_stream( self, - inject_source: Receiver<(Box, oneshot::Sender<()>)>, ) -> impl Stream, Result>> { let notifier = Arc::new(Notify::new()); @@ -215,7 +199,6 @@ impl SourceReader { self.stream_reader, notifier, self.expected_barrier_latency_ms, - inject_source, ); select_with_strategy( barrier_receiver.map(Either::Left), @@ -326,22 +309,21 @@ impl SourceExecutor { }; // todo: use epoch from msg to restore state from state store - let stream_reader = self - .build_stream_source_reader(recover_state) - .await - .map_err(StreamExecutorError::source_error)?; + let stream_reader = Arc::new(Mutex::new( + self.build_stream_source_reader(recover_state) + .await + .map_err(StreamExecutorError::source_error)?, + )); let reader = SourceReader { - stream_reader, + stream_reader: stream_reader.clone(), barrier_receiver, expected_barrier_latency_ms: self.expected_barrier_latency_ms, }; yield Message::Barrier(barrier); - let (inject_source_tx, inject_source_rx) = - channel::<(Box, oneshot::Sender<()>)>(1); #[for_await] - for msg in reader.into_stream(inject_source_rx) { + for msg in reader.into_stream() { match msg { // This branch will be preferred. Either::Left(barrier) => { @@ -370,20 +352,7 @@ impl SourceExecutor { )) .await .map_err(StreamExecutorError::source_error)?; - - let (tx, rx) = oneshot::channel(); - inject_source_tx - .send((reader, tx)) - .await - .to_rw_result() - .map_err(|e| { - StreamExecutorError::channel_closed( - e.to_string(), - ) - })?; - - // force sync - rx.await.unwrap(); + *stream_reader.lock().await = reader; self.stream_source_splits = target_state; } } @@ -473,10 +442,20 @@ mod tests { use std::sync::Arc; use futures::StreamExt; + use maplit::hashmap; use risingwave_common::array::stream_chunk::StreamChunkTestExt; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_common::types::DataType; + use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_connector::datagen::DatagenSplit; + use risingwave_pb::catalog::StreamSourceInfo; + use risingwave_pb::data::data_type::TypeName; + use risingwave_pb::data::DataType as ProstDataType; + use risingwave_pb::plan_common::{ + ColumnCatalog as ProstColumnCatalog, ColumnDesc as ProstColumnDesc, + RowFormatType as ProstRowFormatType, + }; use risingwave_source::*; use risingwave_storage::memory::MemoryStateStore; use tokio::sync::mpsc::unbounded_channel; @@ -706,4 +685,191 @@ mod tests { Ok(()) } + + fn mock_stream_source_info() -> StreamSourceInfo { + let properties: HashMap = hashmap! { + "connector".to_string() => "datagen".to_string(), + "fields.v1.min".to_string() => "1".to_string(), + "fields.v1.max".to_string() => "1000".to_string(), + "fields.v1.seed".to_string() => "12345".to_string(), + }; + + let columns = vec![ + ProstColumnCatalog { + column_desc: Some(ProstColumnDesc { + column_type: Some(ProstDataType { + type_name: TypeName::Int64 as i32, + ..Default::default() + }), + column_id: 0, + ..Default::default() + }), + is_hidden: false, + }, + ProstColumnCatalog { + column_desc: Some(ProstColumnDesc { + column_type: Some(ProstDataType { + type_name: TypeName::Int32 as i32, + ..Default::default() + }), + column_id: 1, + name: "v1".to_string(), + ..Default::default() + }), + is_hidden: false, + }, + ]; + + StreamSourceInfo { + properties, + row_format: ProstRowFormatType::Json as i32, + row_schema_location: "".to_string(), + row_id_index: 0, + columns, + pk_column_ids: vec![0], + } + } + + fn drop_row_id(chunk: StreamChunk) -> StreamChunk { + let (ops, mut columns, bitmap) = chunk.into_inner(); + columns.remove(0); + // columns.pop(); + StreamChunk::new(ops, columns, bitmap) + } + + #[tokio::test] + async fn test_split_change_mutation() -> Result<()> { + let stream_source_info = mock_stream_source_info(); + let source_table_id = TableId::default(); + let source_manager = Arc::new(MemSourceManager::default()); + + source_manager + .create_source(&source_table_id, stream_source_info) + .await?; + + let get_schema = |column_ids: &[ColumnId], source_desc: &SourceDesc| { + let mut fields = Vec::with_capacity(column_ids.len()); + for &column_id in column_ids { + let column_desc = source_desc + .columns + .iter() + .find(|c| c.column_id == column_id) + .unwrap(); + fields.push(Field::unnamed(column_desc.data_type.clone())); + } + Schema::new(fields) + }; + + let actor_id = ActorId::default(); + let source_desc = source_manager.get_source(&source_table_id)?; + let keyspace = Keyspace::table_root(MemoryStateStore::new(), &TableId::from(0x2333)); + let column_ids = vec![ColumnId::from(0), ColumnId::from(1)]; + let schema = get_schema(&column_ids, &source_desc); + let pk_indices = vec![0_usize]; + let (barrier_tx, barrier_rx) = unbounded_channel::(); + + let source_exec = SourceExecutor::new( + actor_id, + source_table_id, + source_desc, + keyspace.clone(), + column_ids.clone(), + schema, + pk_indices, + barrier_rx, + 1, + 1, + "SourceExecutor".to_string(), + Arc::new(StreamingMetrics::unused()), + u64::MAX, + )?; + + let mut materialize = MaterializeExecutor::new( + Box::new(source_exec), + keyspace.clone(), + vec![OrderPair::new(0, OrderType::Ascending)], + column_ids.clone(), + 2, + vec![0usize], + ) + .boxed() + .execute(); + + let curr_epoch = 1919; + let init_barrier = + Barrier::new_test_barrier(curr_epoch).with_mutation(Mutation::AddOutput(AddOutput { + map: HashMap::new(), + splits: hashmap! { + ActorId::default() => vec![ + SplitImpl::Datagen( + DatagenSplit { + split_index: 0, + split_num: 3, + start_offset: None, + }), + ], + }, + })); + barrier_tx.send(init_barrier).unwrap(); + + let _ = materialize.next().await.unwrap(); // barrier + + let chunk_1 = materialize.next().await.unwrap().unwrap().into_chunk(); + + let chunk_1_truth = StreamChunk::from_pretty( + " I i + + 0 533 + + 0 833 + + 0 738 + + 0 344", + ); + + assert_eq!(drop_row_id(chunk_1.unwrap()), drop_row_id(chunk_1_truth)); + + let change_split_mutation = Barrier::new_test_barrier(curr_epoch + 1).with_mutation( + Mutation::SourceChangeSplit(hashmap! { + ActorId::default() => Some(vec![ + SplitImpl::Datagen( + DatagenSplit { + split_index: 0, + split_num: 3, + start_offset: None, + } + ), SplitImpl::Datagen( + DatagenSplit { + split_index: 1, + split_num: 3, + start_offset: None, + } + ), + ]) + }), + ); + barrier_tx.send(change_split_mutation).unwrap(); + + let _ = materialize.next().await.unwrap(); // barrier + + let chunk_2 = materialize.next().await.unwrap().unwrap().into_chunk(); + + let chunk_2_truth = StreamChunk::from_pretty( + " I i + + 0 525 + + 0 425 + + 0 29 + + 0 201", + ); + assert_eq!(drop_row_id(chunk_2.unwrap()), drop_row_id(chunk_2_truth)); + + let chunk_3 = materialize.next().await.unwrap().unwrap().into_chunk(); + + let chunk_3_truth = StreamChunk::from_pretty( + " I i + + 0 833 + + 0 533 + + 0 344", + ); + assert_eq!(drop_row_id(chunk_3.unwrap()), drop_row_id(chunk_3_truth)); + + Ok(()) + } } diff --git a/src/stream/src/from_proto/batch_query.rs b/src/stream/src/from_proto/batch_query.rs index c3317b95684e6..b3566187590fb 100644 --- a/src/stream/src/from_proto/batch_query.rs +++ b/src/stream/src/from_proto/batch_query.rs @@ -13,9 +13,7 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::buffer::Bitmap; -use risingwave_common::catalog::ColumnDesc; -use risingwave_common::hash::VIRTUAL_NODE_COUNT; +use risingwave_common::catalog::{ColumnDesc, OrderedColumnDesc}; use risingwave_storage::table::cell_based_table::CellBasedTable; use risingwave_storage::{Keyspace, StateStore}; @@ -35,7 +33,11 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { let table_id = node.table_desc.as_ref().unwrap().table_id.into(); let pk_descs_proto = &node.table_desc.as_ref().unwrap().order_key; - let pk_descs = pk_descs_proto.iter().map(|d| d.into()).collect(); + let pk_descs = pk_descs_proto + .iter() + .map(OrderedColumnDesc::from) + .collect_vec(); + let order_types = pk_descs.iter().map(|desc| desc.order).collect_vec(); let column_descs = node .column_descs @@ -43,15 +45,14 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { .map(|column_desc| ColumnDesc::from(column_desc.clone())) .collect_vec(); let keyspace = Keyspace::table_root(state_store, &table_id); - let table = CellBasedTable::new(keyspace, column_descs, None, None); + let table = CellBasedTable::new(keyspace, column_descs, order_types, None); let key_indices = node .get_distribution_keys() .iter() .map(|key| *key as usize) .collect_vec(); - let mapping = (*params.vnode_bitmap).clone(); - let hash_filter = Bitmap::from_bytes_with_num_bits(mapping.into(), VIRTUAL_NODE_COUNT); + let hash_filter = params.vnode_bitmap.expect("no vnode bitmap"); let schema = table.schema().clone(); let executor = BatchQueryExecutor::new( diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index e9eb3f61df906..20f6ab7aff7a5 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -14,7 +14,6 @@ use madsim::collections::{HashMap, HashSet}; use risingwave_common::error::Result; -use risingwave_pb::hummock::SstableInfo; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress as ProstCreateMviewProgress; use tokio::sync::mpsc::UnboundedSender; use tokio::sync::oneshot; @@ -29,6 +28,7 @@ mod progress; mod tests; pub use progress::CreateMviewProgress; +use risingwave_hummock_sdk::LocalSstableInfo; /// If enabled, all actors will be grouped in the same tracing span within one epoch. /// Note that this option will significantly increase the overhead of tracing. @@ -39,7 +39,7 @@ pub const ENABLE_BARRIER_AGGREGATION: bool = false; pub struct CollectResult { pub create_mview_progress: Vec, - pub synced_sstables: Vec, + pub synced_sstables: Vec, } enum BarrierState { diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 6a2018c0c5af7..8f9cc6047dca9 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -14,13 +14,13 @@ use core::time::Duration; use std::fmt::Debug; -use std::rc::Rc; use std::sync::Arc; use futures::channel::mpsc::{channel, Receiver}; use itertools::Itertools; use madsim::collections::{HashMap, HashSet}; use parking_lot::Mutex; +use risingwave_common::buffer::Bitmap; use risingwave_common::config::StreamingConfig; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::try_match_expand; @@ -119,7 +119,7 @@ pub struct ExecutorParams { pub actor_context: ActorContextRef, // Vnodes owned by this executor. Represented in bitmap. - pub vnode_bitmap: Rc>, + pub vnode_bitmap: Option, } impl Debug for ExecutorParams { @@ -464,7 +464,7 @@ impl LocalStreamManagerCore { env: StreamEnvironment, store: impl StateStore, actor_context: &ActorContextRef, - vnode_bitmap: Rc>, + vnode_bitmap: Option, ) -> Result { let op_info = node.get_identity().clone(); // Create the input executor before creating itself @@ -482,7 +482,7 @@ impl LocalStreamManagerCore { env.clone(), store.clone(), actor_context, - Rc::clone(&vnode_bitmap), + vnode_bitmap.clone(), ) }) .try_collect()?; @@ -530,7 +530,7 @@ impl LocalStreamManagerCore { node: &stream_plan::StreamNode, env: StreamEnvironment, actor_context: &ActorContextRef, - vnode_bitmap: Rc>, + vnode_bitmap: Option, ) -> Result { dispatch_state_store!(self.state_store.clone(), store, { self.create_nodes_inner( @@ -628,6 +628,11 @@ impl LocalStreamManagerCore { let actor_id = *actor_id; let actor = self.actors.remove(&actor_id).unwrap(); let actor_context = Arc::new(Mutex::new(ActorContext::default())); + let vnode_bitmap = actor + .get_vnode_bitmap() + .ok() + .map(|b| b.try_into()) + .transpose()?; let executor = self.create_nodes( actor.fragment_id, @@ -635,7 +640,7 @@ impl LocalStreamManagerCore { actor.get_nodes()?, env.clone(), &actor_context, - Rc::new(actor.get_vnode_bitmap().to_owned()), + vnode_bitmap, )?; let dispatcher = self.create_dispatcher(executor, &actor.dispatcher, actor_id)?; diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 6224aea3e5c3f..38c1543c9960f 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -35,7 +35,7 @@ hashbrown = { version = "0.11", features = ["ahash", "inline-more", "nightly", " hyper = { version = "0.14", features = ["client", "full", "h2", "http1", "http2", "runtime", "server", "socket2", "stream", "tcp"] } indexmap = { version = "1", default-features = false, features = ["std"] } isahc = { version = "1", default-features = false, features = ["encoding_rs", "mime", "text-decoding"] } -libc = { version = "0.2", features = ["std"] } +libc = { version = "0.2", features = ["extra_traits", "std"] } libz-sys = { version = "1", features = ["libc", "stock-zlib"] } lock_api = { version = "0.4", default-features = false, features = ["arc_lock"] } log = { version = "0.4", default-features = false, features = ["release_max_level_info", "std"] } @@ -91,7 +91,7 @@ hashbrown = { version = "0.11", features = ["ahash", "inline-more", "nightly", " hyper = { version = "0.14", features = ["client", "full", "h2", "http1", "http2", "runtime", "server", "socket2", "stream", "tcp"] } indexmap = { version = "1", default-features = false, features = ["std"] } isahc = { version = "1", default-features = false, features = ["encoding_rs", "mime", "text-decoding"] } -libc = { version = "0.2", features = ["std"] } +libc = { version = "0.2", features = ["extra_traits", "std"] } libz-sys = { version = "1", features = ["libc", "stock-zlib"] } lock_api = { version = "0.4", default-features = false, features = ["arc_lock"] } log = { version = "0.4", default-features = false, features = ["release_max_level_info", "std"] }