From 20e0a98c552dff07279a8b9dee3c2699fba420c1 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 2 Aug 2024 16:26:19 +0800 Subject: [PATCH 1/8] fix(frontend): add exchange for two-phase `approx_percentile` agg (#17900) --- .../tests/testdata/output/agg.yaml | 77 +++++++++++-------- .../src/optimizer/plan_node/logical_agg.rs | 24 +++--- 2 files changed, 55 insertions(+), 46 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 81e4185b128c5..f6fe0ca6b31eb 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1891,9 +1891,10 @@ stream_plan: |- StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: Single } + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with other simple aggs sql: | CREATE TABLE t (v1 int); @@ -1907,10 +1908,11 @@ StreamMaterialize { columns: [approx_percentile, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamKeyedMerge { output: [approx_percentile:Float64, sum(sum(t.v1)):Int64] } ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamExchange { dist: Single } + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } @@ -1931,10 +1933,11 @@ └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum(sum(t.v1)), sum0(count(t.v1))] } └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count(t.v1)):Int64] } ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamExchange { dist: Single } + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), sum0(count(t.v1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v1), count(t.v1)] } @@ -1954,9 +1957,10 @@ StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [approx_percentile, approx_percentile] } └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: Single } + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with different approx_percentile sql: | CREATE TABLE t (v1 int, v2 int); @@ -1971,15 +1975,17 @@ └─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } ├─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } │ ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ │ └─StreamShare { id: 2 } - │ │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } - │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ │ └─StreamExchange { dist: Single } + │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ │ └─StreamShare { id: 2 } + │ │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } + │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } │ └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamExchange { dist: Single } + │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamSimpleAgg { aggs: [count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [] } @@ -2000,15 +2006,17 @@ └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count):Int64, sum(sum(t.v2)):Int64, approx_percentile:Float64] } ├─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } │ ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ │ └─StreamShare { id: 2 } - │ │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } - │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ │ └─StreamExchange { dist: Single } + │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ │ └─StreamShare { id: 2 } + │ │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } + │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } │ └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamExchange { dist: Single } + │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), sum0(count), sum(sum(t.v2)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v1), count, sum(t.v2)] } @@ -2028,10 +2036,11 @@ StreamMaterialize { columns: [s1, approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64] } ├─StreamGlobalApproxPercentile { quantile: 0.8:Float64, relative_error: 0.01:Float64 } - │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.8:Float64, relative_error: 0.01:Float64 } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamExchange { dist: Single } + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.8:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 54c59883b10fb..dc84aa5987159 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -99,7 +99,7 @@ impl LogicalAgg { core.agg_calls = non_approx_percentile_agg_calls; let approx_percentile = - self.build_approx_percentile_aggs(core.input.clone(), &approx_percentile_agg_calls); + self.build_approx_percentile_aggs(core.input.clone(), &approx_percentile_agg_calls)?; // ====== Handle normal aggs let total_agg_calls = core @@ -334,14 +334,14 @@ impl LogicalAgg { &self, input: PlanRef, approx_percentile_agg_call: &PlanAggCall, - ) -> PlanRef { + ) -> Result { let local_approx_percentile = StreamLocalApproxPercentile::new(input, approx_percentile_agg_call); - let global_approx_percentile = StreamGlobalApproxPercentile::new( - local_approx_percentile.into(), - approx_percentile_agg_call, - ); - global_approx_percentile.into() + let exchange = RequiredDist::single() + .enforce_if_not_satisfies(local_approx_percentile.into(), &Order::any())?; + let global_approx_percentile = + StreamGlobalApproxPercentile::new(exchange, approx_percentile_agg_call); + Ok(global_approx_percentile.into()) } /// If only 1 approx percentile, just return it. @@ -361,14 +361,14 @@ impl LogicalAgg { &self, input: PlanRef, approx_percentile_agg_call: &[PlanAggCall], - ) -> Option { + ) -> Result> { if approx_percentile_agg_call.is_empty() { - return None; + return Ok(None); } - let approx_percentile_plans = approx_percentile_agg_call + let approx_percentile_plans: Vec = approx_percentile_agg_call .iter() .map(|agg_call| self.build_approx_percentile_agg(input.clone(), agg_call)) - .collect_vec(); + .try_collect()?; assert!(!approx_percentile_plans.is_empty()); let mut iter = approx_percentile_plans.into_iter(); let mut acc = iter.next().unwrap(); @@ -383,7 +383,7 @@ impl LogicalAgg { .expect("failed to build keyed merge"); acc = keyed_merge.into(); } - Some(acc) + Ok(Some(acc)) } pub fn core(&self) -> &Agg { From 911578c0a8700ee7a284480dacb8f8e07aa510d0 Mon Sep 17 00:00:00 2001 From: WanYixian <150207222+WanYixian@users.noreply.github.com> Date: Fri, 2 Aug 2024 16:38:23 +0800 Subject: [PATCH 2/8] chore: Improve wording of error message (#17766) Co-authored-by: Richard Chien Co-authored-by: Eric Fu --- Makefile.toml | 2 +- e2e_test/source/basic/ddl.slt | 2 +- e2e_test/source/basic/old_row_format_syntax/ddl.slt | 2 +- .../java/com/risingwave/connector/CassandraUtil.java | 8 ++++---- .../risingwave/mock/flink/common/FlinkDynamicUtil.java | 5 +++-- src/connector/src/connector_common/common.rs | 2 +- src/connector/src/sink/big_query.rs | 4 ++-- src/connector/src/sink/deltalake.rs | 4 ++-- src/connector/src/sink/elasticsearch.rs | 8 ++++---- src/connector/src/sink/iceberg/mod.rs | 8 ++++---- src/connector/src/sink/log_store.rs | 4 ++-- src/connector/src/sink/mqtt.rs | 6 +++--- src/connector/src/sink/nats.rs | 4 ++-- src/connector/src/sink/redis.rs | 4 ++-- src/connector/src/sink/remote.rs | 10 +++++----- src/connector/src/sink/starrocks.rs | 2 +- src/connector/src/source/kafka/enumerator/client.rs | 2 +- src/connector/src/source/kinesis/source/reader.rs | 4 ++-- src/connector/src/source/pulsar/enumerator/client.rs | 2 +- src/meta/src/controller/catalog.rs | 2 +- src/meta/src/controller/fragment.rs | 2 +- src/meta/src/manager/catalog/database.rs | 2 +- src/meta/src/manager/catalog/fragment.rs | 2 +- src/meta/src/manager/catalog/mod.rs | 2 +- .../manager/sink_coordination/coordinator_worker.rs | 7 ++++--- src/meta/src/rpc/ddl_controller.rs | 2 +- src/meta/src/stream/stream_graph/actor.rs | 2 +- src/risedevtool/src/config/id_expander.rs | 2 +- src/risedevtool/src/config_gen/grafana_gen.rs | 8 ++++---- .../src/common/log_store_impl/kv_log_store/serde.rs | 2 +- src/stream/src/executor/source/state_table_handler.rs | 2 +- src/stream/src/executor/watermark_filter.rs | 2 +- src/tests/regress/src/schedule.rs | 6 +++--- .../tests/integration_tests/recovery/background_ddl.rs | 2 +- 34 files changed, 65 insertions(+), 63 deletions(-) diff --git a/Makefile.toml b/Makefile.toml index 5f8a2ef648293..2fe13db1ca8cc 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -369,7 +369,7 @@ ln -s "$(pwd)/target/${RISEDEV_BUILD_TARGET_DIR}${BUILD_MODE_DIR}/risingwave" "$ [tasks.post-build-risingwave] category = "RiseDev - Build" -description = "Copy RisngWave binaries to bin" +description = "Copy RisingWave binaries to bin" condition = { env_set = ["ENABLE_BUILD_RUST"] } dependencies = [ "link-all-in-one-binaries", diff --git a/e2e_test/source/basic/ddl.slt b/e2e_test/source/basic/ddl.slt index a56d90934c149..8e63971fb5b82 100644 --- a/e2e_test/source/basic/ddl.slt +++ b/e2e_test/source/basic/ddl.slt @@ -53,7 +53,7 @@ create source s ( properties.bootstrap.server = 'message_queue:29092' ) FORMAT PLAIN ENCODE JSON; -statement error properties `scan_startup_mode` only support earliest and latest or leave it empty +statement error properties `scan_startup_mode` only supports earliest and latest or leaving it empty create source invalid_startup_mode ( column1 varchar ) with ( diff --git a/e2e_test/source/basic/old_row_format_syntax/ddl.slt b/e2e_test/source/basic/old_row_format_syntax/ddl.slt index b48249ca7e393..0fe67a8504b5d 100644 --- a/e2e_test/source/basic/old_row_format_syntax/ddl.slt +++ b/e2e_test/source/basic/old_row_format_syntax/ddl.slt @@ -7,7 +7,7 @@ create source s ( properties.bootstrap.server = 'message_queue:29092' ) ROW FORMAT JSON; -statement error properties `scan_startup_mode` only support earliest and latest or leave it empty +statement error properties `scan_startup_mode` only supports earliest and latest or leaving it empty create source invalid_startup_mode ( column1 varchar ) with ( diff --git a/java/connector-node/risingwave-sink-cassandra/src/main/java/com/risingwave/connector/CassandraUtil.java b/java/connector-node/risingwave-sink-cassandra/src/main/java/com/risingwave/connector/CassandraUtil.java index 8327893f6da9a..463bdb9d3f113 100644 --- a/java/connector-node/risingwave-sink-cassandra/src/main/java/com/risingwave/connector/CassandraUtil.java +++ b/java/connector-node/risingwave-sink-cassandra/src/main/java/com/risingwave/connector/CassandraUtil.java @@ -91,7 +91,7 @@ public static void checkSchema( throw Status.FAILED_PRECONDITION .withDescription( String.format( - "Don't match in the name, rw is %s cassandra can't find it", + "Name mismatch. Column `%s` on RisingWave side is not found on Cassandra side.", columnDesc.getName())) .asRuntimeException(); } @@ -100,7 +100,7 @@ public static void checkSchema( throw Status.FAILED_PRECONDITION .withDescription( String.format( - "Don't match in the type, name is %s, cassandra is %s, rw is %s", + "Data type mismatch for column `%s`. Cassandra side: `%s`, RisingWave side: `%s`.", columnDesc.getName(), cassandraColumnDescMap.get(cql), columnDesc.getDataType().getTypeName())) @@ -113,7 +113,7 @@ public static void checkPrimaryKey( List cassandraColumnMetadatas, List columnMetadatas) { if (cassandraColumnMetadatas.size() != columnMetadatas.size()) { throw Status.FAILED_PRECONDITION - .withDescription("Primary key len don't match") + .withDescription("Primary key length mismatch.") .asRuntimeException(); } Set cassandraColumnsSet = @@ -125,7 +125,7 @@ public static void checkPrimaryKey( throw Status.FAILED_PRECONDITION .withDescription( String.format( - "Primary key don't match. RisingWave Primary key is %s, don't find it in cassandra", + "Primary key mismatch. Primary key `%s` on RisingWave side is not found on Cassandra side", columnMetadata)) .asRuntimeException(); } diff --git a/java/connector-node/risingwave-sink-mock-flink/risingwave-sink-mock-flink-common/src/main/java/com/risingwave/mock/flink/common/FlinkDynamicUtil.java b/java/connector-node/risingwave-sink-mock-flink/risingwave-sink-mock-flink-common/src/main/java/com/risingwave/mock/flink/common/FlinkDynamicUtil.java index 280b6b7eccb44..4a58c3160b075 100644 --- a/java/connector-node/risingwave-sink-mock-flink/risingwave-sink-mock-flink-common/src/main/java/com/risingwave/mock/flink/common/FlinkDynamicUtil.java +++ b/java/connector-node/risingwave-sink-mock-flink/risingwave-sink-mock-flink-common/src/main/java/com/risingwave/mock/flink/common/FlinkDynamicUtil.java @@ -120,14 +120,15 @@ public static void validateSchemaWithCatalog( throw Status.FAILED_PRECONDITION .withDescription( String.format( - "Don't match in the name, rw is %s", columnDesc.getName())) + "Name mismatch. Column `%s` on RisingWave side is not found on Flink side.", + columnDesc.getName())) .asRuntimeException(); } if (!checkType(columnDesc.getDataType(), flinkColumnMap.get(columnDesc.getName()))) { throw Status.FAILED_PRECONDITION .withDescription( String.format( - "Don't match in the type, name is %s, Sink is %s, rw is %s", + "Data type mismatch for column `%s`. Flink side: `%s`, RisingWave side: `%s`.", columnDesc.getName(), flinkColumnMap.get(columnDesc.getName()), columnDesc.getDataType().getTypeName())) diff --git a/src/connector/src/connector_common/common.rs b/src/connector/src/connector_common/common.rs index 92bb7d9c30677..0933f38dc14e0 100644 --- a/src/connector/src/connector_common/common.rs +++ b/src/connector/src/connector_common/common.rs @@ -601,7 +601,7 @@ impl NatsCommon { } "plain" => {} _ => { - bail!("nats connect mode only accept user_and_password/credential/plain"); + bail!("nats connect mode only accepts user_and_password/credential/plain"); } }; diff --git a/src/connector/src/sink/big_query.rs b/src/connector/src/sink/big_query.rs index 0535c03250563..bbe0139caaa46 100644 --- a/src/connector/src/sink/big_query.rs +++ b/src/connector/src/sink/big_query.rs @@ -211,14 +211,14 @@ impl BigQuerySink { for i in rw_fields_name { let value = big_query_columns_desc.get(&i.name).ok_or_else(|| { SinkError::BigQuery(anyhow::anyhow!( - "Column name don't find in bigquery, risingwave is {:?} ", + "Column `{:?}` on RisingWave side is not found on BigQuery side.", i.name )) })?; let data_type_string = Self::get_string_and_check_support_from_datatype(&i.data_type)?; if data_type_string.ne(value) { return Err(SinkError::BigQuery(anyhow::anyhow!( - "Column type don't match, column name is {:?}. bigquery type is {:?} risingwave type is {:?} ",i.name,value,data_type_string + "Data type mismatch for column `{:?}`. BigQuery side: `{:?}`, RisingWave side: `{:?}`. ", i.name, value, data_type_string ))); }; } diff --git a/src/connector/src/sink/deltalake.rs b/src/connector/src/sink/deltalake.rs index 43e9d2c98cf6c..661145959a0ff 100644 --- a/src/connector/src/sink/deltalake.rs +++ b/src/connector/src/sink/deltalake.rs @@ -135,7 +135,7 @@ impl DeltaLakeCommon { Ok(DeltaTableUrl::Local(path.to_string())) } else { Err(SinkError::DeltaLake(anyhow!( - "path need to start with 's3://','s3a://'(s3) ,gs://(gcs) or file://(local)" + "path should start with 's3://','s3a://'(s3) ,gs://(gcs) or file://(local)" ))) } } @@ -356,7 +356,7 @@ impl Sink for DeltaLakeSink { .collect(); if deltalake_fields.len() != self.param.schema().fields().len() { return Err(SinkError::DeltaLake(anyhow!( - "column count not match, rw is {}, deltalake is {}", + "Columns mismatch. RisingWave is {}, DeltaLake is {}", self.param.schema().fields().len(), deltalake_fields.len() ))); diff --git a/src/connector/src/sink/elasticsearch.rs b/src/connector/src/sink/elasticsearch.rs index 3d51e48201c94..31dde5c52509e 100644 --- a/src/connector/src/sink/elasticsearch.rs +++ b/src/connector/src/sink/elasticsearch.rs @@ -87,7 +87,7 @@ impl EsStreamChunkConverter { Box::new(|row: RowRef<'_>| { Ok(row .datum_at(0) - .ok_or_else(|| anyhow!("No value find in row, index is 0"))? + .ok_or_else(|| anyhow!("No value found in row, index is 0"))? .to_text()) }) } else if pk_indices.len() == 1 { @@ -95,7 +95,7 @@ impl EsStreamChunkConverter { Box::new(move |row: RowRef<'_>| { Ok(row .datum_at(index) - .ok_or_else(|| anyhow!("No value find in row, index is 0"))? + .ok_or_else(|| anyhow!("No value found in row, index is 0"))? .to_text()) }) } else { @@ -108,7 +108,7 @@ impl EsStreamChunkConverter { for index in &pk_indices { keys.push( row.datum_at(*index) - .ok_or_else(|| anyhow!("No value find in row, index is {}", index))? + .ok_or_else(|| anyhow!("No value found in row, index is {}", index))? .to_text(), ); } @@ -144,7 +144,7 @@ impl EsStreamChunkConverter { if let Some(index) = self.index_column { index_builder.append(Some( row.datum_at(index) - .ok_or_else(|| anyhow!("No value find in row, index is {}", index))? + .ok_or_else(|| anyhow!("No value found in row, index is {}", index))? .into_utf8(), )); } else { diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index 4f4af102406c1..76c9dff491371 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -712,7 +712,7 @@ impl Sink for IcebergSink { self.param.clone(), writer_param.vnode_bitmap.ok_or_else(|| { SinkError::Remote(anyhow!( - "sink needs coordination should not have singleton input" + "sink needs coordination and should not have singleton input" )) })?, inner, @@ -1028,7 +1028,7 @@ impl WriteResult { { v } else { - bail!("iceberg sink metadata should be a object"); + bail!("iceberg sink metadata should be an object"); }; let data_files: Vec; @@ -1055,7 +1055,7 @@ impl WriteResult { .collect::, icelake::Error>>() .context("Failed to parse data file from json")?; } else { - bail!("icberg sink metadata should have data_files object"); + bail!("Iceberg sink metadata should have data_files object"); } Ok(Self { data_files, @@ -1155,7 +1155,7 @@ pub fn try_matches_arrow_schema( ) -> anyhow::Result<()> { if rw_schema.fields.len() != arrow_schema.fields().len() { bail!( - "Schema length not match, risingwave is {}, and iceberg is {}", + "Schema length mismatch, risingwave is {}, and iceberg is {}", rw_schema.fields.len(), arrow_schema.fields.len() ); diff --git a/src/connector/src/sink/log_store.rs b/src/connector/src/sink/log_store.rs index 346792d440d48..3c25b1ec7d75a 100644 --- a/src/connector/src/sink/log_store.rs +++ b/src/connector/src/sink/log_store.rs @@ -85,7 +85,7 @@ impl TruncateOffset { } => { if epoch != *offset_epoch { bail!( - "new item epoch {} not match current chunk offset epoch {}", + "new item epoch {} does not match current chunk offset epoch {}", epoch, offset_epoch ); @@ -96,7 +96,7 @@ impl TruncateOffset { } => { if epoch <= *offset_epoch { bail!( - "new item epoch {} not exceed barrier offset epoch {}", + "new item epoch {} does not exceed barrier offset epoch {}", epoch, offset_epoch ); diff --git a/src/connector/src/sink/mqtt.rs b/src/connector/src/sink/mqtt.rs index 072666b015641..896da3a7f4d89 100644 --- a/src/connector/src/sink/mqtt.rs +++ b/src/connector/src/sink/mqtt.rs @@ -133,7 +133,7 @@ impl MqttConfig { .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY { Err(SinkError::Config(anyhow!( - "Mqtt sink only support append-only mode" + "MQTT sink only supports append-only mode" ))) } else { Ok(config) @@ -175,7 +175,7 @@ impl Sink for MqttSink { async fn validate(&self) -> Result<()> { if !self.is_append_only { return Err(SinkError::Mqtt(anyhow!( - "Mqtt sink only support append-only mode" + "MQTT sink only supports append-only mode" ))); } @@ -261,7 +261,7 @@ impl MqttSinkWriter { }, _ => { return Err(SinkError::Config(anyhow!( - "Mqtt sink only support append-only mode" + "MQTT sink only supports append-only mode" ))) } }; diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index 5ec68ab1c5c63..878cc1bee27d2 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -79,7 +79,7 @@ impl NatsConfig { .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY { Err(SinkError::Config(anyhow!( - "Nats sink only support append-only mode" + "NATS sink only supports append-only mode" ))) } else { Ok(config) @@ -117,7 +117,7 @@ impl Sink for NatsSink { async fn validate(&self) -> Result<()> { if !self.is_append_only { return Err(SinkError::Nats(anyhow!( - "Nats sink only support append-only mode" + "NATS sink only supports append-only mode" ))); } let _client = (self.config.common.build_client().await) diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index df972ef7ba392..49207e668e41b 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -233,12 +233,12 @@ impl Sink for RedisSink { ) { let key_format = self.format_desc.options.get(KEY_FORMAT).ok_or_else(|| { SinkError::Config(anyhow!( - "Cannot find 'key_format',please set it or use JSON" + "Cannot find 'key_format', please set it or use JSON" )) })?; let value_format = self.format_desc.options.get(VALUE_FORMAT).ok_or_else(|| { SinkError::Config(anyhow!( - "Cannot find 'value_format',please set it or use JSON" + "Cannot find 'value_format', please set it or use JSON" )) })?; TemplateEncoder::check_string_format(key_format, &pk_set)?; diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index d0edded78d6db..23ea54944f11c 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -169,7 +169,7 @@ async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> ConnectorRe && param.downstream_pk.len() > 1 && !param.properties.contains_key(ES_OPTION_DELIMITER) { - bail!("Es sink only support single pk or pk with delimiter option"); + bail!("Es sink only supports single pk or pk with delimiter option"); } // FIXME: support struct and array in stream sink param.columns.iter().map(|col| { @@ -194,7 +194,7 @@ async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> ConnectorRe Ok(()) } else{ Err(SinkError::Remote(anyhow!( - "Remote sink only support list, got {:?}: {:?}", + "Remote sink only supports list, got {:?}: {:?}", col.name, col.data_type, ))) @@ -205,7 +205,7 @@ async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> ConnectorRe Ok(()) }else{ Err(SinkError::Remote(anyhow!( - "Only Es sink support struct, got {:?}: {:?}", + "Only Es sink supports struct, got {:?}: {:?}", col.name, col.data_type, ))) @@ -347,7 +347,7 @@ impl LogSinker for RemoteLogSinker { })?; if sent_offset != persisted_offset { bail!( - "new response offset {:?} not match the buffer offset {:?}", + "new response offset {:?} does not match the buffer offset {:?}", persisted_offset, sent_offset ); @@ -535,7 +535,7 @@ impl Sink for CoordinatedRemoteSink { self.param.clone(), writer_param.vnode_bitmap.ok_or_else(|| { SinkError::Remote(anyhow!( - "sink needs coordination should not have singleton input" + "sink needs coordination and should not have singleton input" )) })?, CoordinatedRemoteSinkWriter::new(self.param.clone(), writer_param.sink_metrics.clone()) diff --git a/src/connector/src/sink/starrocks.rs b/src/connector/src/sink/starrocks.rs index 56c352bfb4a9e..e5881ee9f747e 100644 --- a/src/connector/src/sink/starrocks.rs +++ b/src/connector/src/sink/starrocks.rs @@ -340,7 +340,7 @@ impl Sink for StarrocksSink { self.param.clone(), writer_param.vnode_bitmap.ok_or_else(|| { SinkError::Remote(anyhow!( - "sink needs coordination should not have singleton input" + "sink needs coordination and should not have singleton input" )) })?, inner, diff --git a/src/connector/src/source/kafka/enumerator/client.rs b/src/connector/src/source/kafka/enumerator/client.rs index 82dfaf11e5d83..ff007076c1338 100644 --- a/src/connector/src/source/kafka/enumerator/client.rs +++ b/src/connector/src/source/kafka/enumerator/client.rs @@ -82,7 +82,7 @@ impl SplitEnumerator for KafkaSplitEnumerator { Some("latest") => KafkaEnumeratorOffset::Latest, None => KafkaEnumeratorOffset::Earliest, _ => bail!( - "properties `scan_startup_mode` only support earliest and latest or leave it empty" + "properties `scan_startup_mode` only supports earliest and latest or leaving it empty" ), }; diff --git a/src/connector/src/source/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index b728270dbd821..56a8b70d301a4 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -91,7 +91,7 @@ impl SplitReader for KinesisSplitReader { if !matches!(start_position, KinesisOffset::Timestamp(_)) && properties.timestamp_offset.is_some() { - bail!("scan.startup.mode need to be set to 'timestamp' if you want to start with a specific timestamp"); + bail!("scan.startup.mode needs to be set to 'timestamp' if you want to start with a specific timestamp"); } let stream_name = properties.common.stream_name.clone(); @@ -199,7 +199,7 @@ impl KinesisSplitReader { } Err(e) => { let error = anyhow!(e).context(format!( - "Kinesis got a unhandled error on stream {:?}, shard {:?}", + "Kinesis got an unhandled error on stream {:?}, shard {:?}", self.stream_name, self.shard_id )); tracing::error!(error = %error.as_report()); diff --git a/src/connector/src/source/pulsar/enumerator/client.rs b/src/connector/src/source/pulsar/enumerator/client.rs index dddcf927d0c3f..ef79a8f576d3e 100644 --- a/src/connector/src/source/pulsar/enumerator/client.rs +++ b/src/connector/src/source/pulsar/enumerator/client.rs @@ -65,7 +65,7 @@ impl SplitEnumerator for PulsarSplitEnumerator { None => PulsarEnumeratorOffset::Earliest, _ => { bail!( - "properties `startup_mode` only support earliest and latest or leave it empty" + "properties `startup_mode` only supports earliest and latest or leaving it empty" ); } }; diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 4ee0a018f5b92..8308d7a13e193 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -2683,7 +2683,7 @@ impl CatalogController { .into_iter() .map(|(subscription, obj)| ObjectModel(subscription, obj.unwrap()).into()) .find_or_first(|_| true) - .ok_or_else(|| anyhow!("cant find subscription with id {}", subscription_id))?; + .ok_or_else(|| anyhow!("cannot find subscription with id {}", subscription_id))?; Ok(subscription) } diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 4b2d0bd162dcf..6c24a6a44e5b4 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -209,7 +209,7 @@ impl CatalogController { for mut actor in pb_actors { let mut upstream_actors = BTreeMap::new(); - let node = actor.nodes.as_mut().context("nodes is empty")?; + let node = actor.nodes.as_mut().context("nodes are empty")?; visit_stream_node(node, |body| { if let NodeBody::Merge(m) = body { diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index e3b2e0d02cae7..da1238d43d54b 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -255,7 +255,7 @@ impl DatabaseManager { && x.name.eq(&relation_key.2) }) { if t.stream_job_status == StreamJobStatus::Creating as i32 { - bail!("table is in creating procedure, table id: {}", t.id); + bail!("The table is being created, table id: {}", t.id); } else { Err(MetaError::catalog_duplicated("table", &relation_key.2)) } diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index eab1da29f4d7c..42e80d1c13a64 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -347,7 +347,7 @@ impl FragmentManager { let map = &mut guard.table_fragments; let table_id = table_fragment.table_id(); if map.contains_key(&table_id) { - bail!("table_fragment already exist: id={}", table_id); + bail!("table_fragment already exists: id={}", table_id); } let mut table_fragments = BTreeMapTransaction::new(map); diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 62ecb766a2d74..2997b0cc3b164 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -1009,7 +1009,7 @@ impl CatalogManager { database_core.check_relation_name_duplicated(&key)?; if database_core.has_in_progress_creation(&key) { - bail!("table is in creating procedure"); + bail!("The table is being created"); } else { database_core.mark_creating(&key); database_core.mark_creating_streaming_job(table.id, key); diff --git a/src/meta/src/manager/sink_coordination/coordinator_worker.rs b/src/meta/src/manager/sink_coordination/coordinator_worker.rs index 679a7479419e3..8409e714852c2 100644 --- a/src/meta/src/manager/sink_coordination/coordinator_worker.rs +++ b/src/meta/src/manager/sink_coordination/coordinator_worker.rs @@ -153,7 +153,7 @@ impl CoordinatorWorker { "one sink writer stream reaches the end before initialize" )), Either::Right((Some(Err(e)), _)) => { - Err(anyhow!(e).context("unable to poll from one sink writer stream")) + Err(anyhow!(e).context("unable to poll one sink writer stream")) } Either::Right((None, _)) => unreachable!("request_streams must not be empty"), } @@ -267,8 +267,9 @@ impl CoordinatorWorker { )); } Err(e) => { - return Err(anyhow!(e) - .context("failed to poll from one of the writer request streams")); + return Err( + anyhow!(e).context("failed to poll one of the writer request streams") + ); } }, Either::Right((None, _)) => { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 3dd1e8401c226..c5a750ee7230a 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -388,7 +388,7 @@ impl DdlController { .await? .is_empty() { - bail!("There are background creating jobs, please try again later") + bail!("The system is creating jobs in the background, please try again later") } self.stream_manager diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index cea2e8d4c0c15..8847ff1e8c028 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -740,7 +740,7 @@ impl ActorGraphBuilder { .contains(&worker_slot_id.worker_id()) { bail!( - "The worker {} where the associated upstream is located is unscheduable", + "The worker {} where the associated upstream is located is unschedulable", worker_slot_id.worker_id(), ); } diff --git a/src/risedevtool/src/config/id_expander.rs b/src/risedevtool/src/config/id_expander.rs index 01eae93843f5c..2c9f35cdee6ac 100644 --- a/src/risedevtool/src/config/id_expander.rs +++ b/src/risedevtool/src/config/id_expander.rs @@ -36,7 +36,7 @@ impl IdExpander { .ok_or_else(|| anyhow!("Id isn't a string: {:?}", item))?; ids.push(id.to_string()); } else { - return Err(anyhow!("Not an hashmap: {:?}", item)); + return Err(anyhow!("Not a hashmap: {:?}", item)); } } Ok(Self { ids }) diff --git a/src/risedevtool/src/config_gen/grafana_gen.rs b/src/risedevtool/src/config_gen/grafana_gen.rs index b326053e22737..1977be134fff6 100644 --- a/src/risedevtool/src/config_gen/grafana_gen.rs +++ b/src/risedevtool/src/config_gen/grafana_gen.rs @@ -50,7 +50,7 @@ org_role = Admin let provide_prometheus = config.provide_prometheus.as_ref().unwrap(); if provide_prometheus.len() != 1 { return Err(anyhow!( - "expect 1 prometheus nodes, found {}", + "expect 1 prometheus node, found {}", provide_prometheus.len() )); } @@ -84,7 +84,7 @@ datasources: let provide_tempo = config.provide_tempo.as_ref().unwrap(); if provide_tempo.len() != 1 { return Err(anyhow!( - "expect 1 tempo nodes, found {}", + "expect 1 tempo node, found {}", provide_tempo.len() )); } @@ -120,7 +120,7 @@ datasources: let provide_prometheus = config.provide_prometheus.as_ref().unwrap(); if provide_prometheus.len() != 1 { return Err(anyhow!( - "expect 1 prometheus nodes, found {}", + "expect 1 prometheus node, found {}", provide_prometheus.len() )); }; @@ -170,7 +170,7 @@ providers: let provide_prometheus = config.provide_prometheus.as_ref().unwrap(); if provide_prometheus.len() != 1 { return Err(anyhow!( - "expect 1 prometheus nodes, found {}", + "expect 1 prometheus node, found {}", provide_prometheus.len() )); }; diff --git a/src/stream/src/common/log_store_impl/kv_log_store/serde.rs b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs index f576d688f20e8..92a3caf4cd2e3 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/serde.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs @@ -835,7 +835,7 @@ impl LogStoreRowOpStream { } if *stream_epoch < epoch { return Err(anyhow!( - "current epoch {} has exceed epoch {} of stream not started", + "current epoch {} has exceeded the epoch {} of the stream that has not started", epoch, stream_epoch )); diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index 590401de1847d..1c9615c542bfe 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -144,7 +144,7 @@ impl SourceStateTableHandler { ) -> StreamExecutorResult<()> { if states.is_empty() { // TODO should be a clear Error Code - bail!("states require not null"); + bail!("states should not be null"); } else { for split in states { self.set_complete(split.id(), split.encode_to_json()) diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index cf05464917674..8f8b166626d21 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -333,7 +333,7 @@ impl WatermarkFilterExecutor { if row.len() == 1 { Ok::<_, StreamExecutorError>(row[0].to_owned()) } else { - bail!("The watermark row should only contains 1 datum"); + bail!("The watermark row should only contain 1 datum"); } } _ => Ok(None), diff --git a/src/tests/regress/src/schedule.rs b/src/tests/regress/src/schedule.rs index 269a0881ab3d0..0c599ec5a3dfb 100644 --- a/src/tests/regress/src/schedule.rs +++ b/src/tests/regress/src/schedule.rs @@ -140,15 +140,15 @@ impl Schedule { if !different_tests.is_empty() { info!( - "Risingwave regress tests failed, these tests are different from expected output: {:?}", + "RisingWave regress tests failed, these tests are different from expected output: {:?}", different_tests ); bail!( - "Risingwave regress tests failed, these tests are different from expected output: {:?}", + "RisingWave regress tests failed, these tests are different from expected output: {:?}", different_tests ) } else { - info!("Risingwave regress tests passed."); + info!("RisingWave regress tests passed."); Ok(()) } } diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs index b3776bcd57b64..c5a642998c513 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -229,7 +229,7 @@ async fn test_ddl_cancel() -> Result<()> { let result = create_mv(&mut session).await; match result { Ok(_) => break, - Err(e) if e.to_string().contains("in creating procedure") => { + Err(e) if e.to_string().contains("The table is being created") => { tracing::info!("create mv failed, retrying: {}", e); } Err(e) => { From c09d2647ef245d214da3e5b1826ae593a0b16d41 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 2 Aug 2024 18:32:23 +0800 Subject: [PATCH 3/8] feat(cdc): parse debezium schema event for mysql (#17707) --- .../connector/source/core/DbzCdcEngine.java | 1 + .../source/core/DbzChangeEventConsumer.java | 67 +++++- proto/connector_service.proto | 11 +- proto/ddl_service.proto | 18 ++ src/common/src/types/jsonb.rs | 8 + src/connector/codec/src/decoder/mod.rs | 3 + src/connector/src/parser/debezium/mod.rs | 2 + .../src/parser/debezium/schema_change.rs | 94 ++++++++ .../src/parser/debezium/simd_json_parser.rs | 7 + src/connector/src/parser/mod.rs | 8 + src/connector/src/parser/plain_parser.rs | 210 ++++++++++++++++-- src/connector/src/parser/unified/debezium.rs | 98 ++++++++ .../src/source/cdc/external/mysql.rs | 53 ++++- .../src/source/cdc/source/message.rs | 38 +++- 14 files changed, 585 insertions(+), 33 deletions(-) create mode 100644 src/connector/src/parser/debezium/schema_change.rs diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java index ed22fe36416ec..9227d8225a65c 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java @@ -47,6 +47,7 @@ public DbzCdcEngine( sourceId, heartbeatTopicPrefix, transactionTopic, + topicPrefix, new ArrayBlockingQueue<>(DEFAULT_QUEUE_CAPACITY)); // Builds a debezium engine but not start it diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java index 98a0a171ec4cc..375b4d4a3ad62 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java @@ -17,6 +17,7 @@ import com.risingwave.connector.api.source.SourceTypeE; import com.risingwave.connector.cdc.debezium.internal.DebeziumOffset; import com.risingwave.connector.cdc.debezium.internal.DebeziumOffsetSerializer; +import com.risingwave.connector.source.common.CdcConnectorException; import com.risingwave.proto.ConnectorServiceProto.CdcMessage; import com.risingwave.proto.ConnectorServiceProto.GetEventStreamResponse; import io.debezium.connector.postgresql.PostgresOffsetContext; @@ -43,6 +44,7 @@ enum EventType { HEARTBEAT, TRANSACTION, DATA, + SCHEMA_CHANGE, } public class DbzChangeEventConsumer @@ -57,6 +59,7 @@ public class DbzChangeEventConsumer private final JsonConverter keyConverter; private final String heartbeatTopicPrefix; private final String transactionTopic; + private final String schemaChangeTopic; private volatile DebeziumEngine.RecordCommitter> currentRecordCommitter; @@ -66,12 +69,14 @@ public class DbzChangeEventConsumer long sourceId, String heartbeatTopicPrefix, String transactionTopic, + String schemaChangeTopic, BlockingQueue queue) { this.connector = connector; this.sourceId = sourceId; this.outputChannel = queue; this.heartbeatTopicPrefix = heartbeatTopicPrefix; this.transactionTopic = transactionTopic; + this.schemaChangeTopic = schemaChangeTopic; LOG.info("heartbeat topic: {}, trnx topic: {}", heartbeatTopicPrefix, transactionTopic); // The default JSON converter will output the schema field in the JSON which is unnecessary @@ -105,6 +110,8 @@ private EventType getEventType(SourceRecord record) { return EventType.HEARTBEAT; } else if (isTransactionMetaEvent(record)) { return EventType.TRANSACTION; + } else if (isSchemaChangeEvent(record)) { + return EventType.SCHEMA_CHANGE; } else { return EventType.DATA; } @@ -122,6 +129,11 @@ private boolean isTransactionMetaEvent(SourceRecord record) { return topic != null && topic.equals(transactionTopic); } + private boolean isSchemaChangeEvent(SourceRecord record) { + String topic = record.topic(); + return topic != null && topic.equals(schemaChangeTopic); + } + @Override public void handleBatch( List> events, @@ -155,7 +167,8 @@ var record = event.value(); switch (eventType) { case HEARTBEAT: { - var message = msgBuilder.build(); + var message = + msgBuilder.setMsgType(CdcMessage.CdcMessageType.HEARTBEAT).build(); LOG.debug("heartbeat => {}", message.getOffset()); respBuilder.addEvents(message); break; @@ -168,7 +181,7 @@ var record = event.value(); record.topic(), record.valueSchema(), record.value()); var message = msgBuilder - .setIsTransactionMeta(true) + .setMsgType(CdcMessage.CdcMessageType.TRANSACTION_META) .setPayload(new String(payload, StandardCharsets.UTF_8)) .setSourceTsMs(trxTs) .build(); @@ -176,6 +189,46 @@ var record = event.value(); respBuilder.addEvents(message); break; } + + case SCHEMA_CHANGE: + { + var sourceStruct = ((Struct) record.value()).getStruct("source"); + if (sourceStruct == null) { + throw new CdcConnectorException( + "source field is missing in schema change event"); + } + + // upstream event time + long sourceTsMs = sourceStruct.getInt64("ts_ms"); + byte[] payload = + payloadConverter.fromConnectData( + record.topic(), record.valueSchema(), record.value()); + + // We intentionally don't set the fullTableName for schema change event, + // since it doesn't need to be routed to a specific cdc table + var message = + msgBuilder + .setMsgType(CdcMessage.CdcMessageType.SCHEMA_CHANGE) + .setPayload(new String(payload, StandardCharsets.UTF_8)) + .setSourceTsMs(sourceTsMs) + .build(); + LOG.debug( + "offset => {}, key => {}, payload => {}", + message.getOffset(), + message.getKey(), + message.getPayload()); + respBuilder.addEvents(message); + + // emit the schema change event as a single response + respBuilder.setSourceId(sourceId); + var response = respBuilder.build(); + outputChannel.put(response); + + // reset the response builder + respBuilder = GetEventStreamResponse.newBuilder(); + break; + } + case DATA: { // Topic naming conventions @@ -192,10 +245,11 @@ var record = event.value(); } // get upstream event time from the "source" field var sourceStruct = ((Struct) record.value()).getStruct("source"); - long sourceTsMs = - sourceStruct == null - ? System.currentTimeMillis() - : sourceStruct.getInt64("ts_ms"); + if (sourceStruct == null) { + throw new CdcConnectorException( + "source field is missing in data change event"); + } + long sourceTsMs = sourceStruct.getInt64("ts_ms"); byte[] payload = payloadConverter.fromConnectData( record.topic(), record.valueSchema(), record.value()); @@ -208,6 +262,7 @@ var record = event.value(); String msgKey = key == null ? "" : new String(key, StandardCharsets.UTF_8); var message = msgBuilder + .setMsgType(CdcMessage.CdcMessageType.DATA) .setFullTableName(fullTableName) .setPayload(msgPayload) .setKey(msgKey) diff --git a/proto/connector_service.proto b/proto/connector_service.proto index da2c2b88087ea..cf549a8e2e493 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -147,13 +147,22 @@ message SinkCoordinatorStreamResponse { /* Source Service */ message CdcMessage { + enum CdcMessageType { + UNSPECIFIED = 0; + HEARTBEAT = 1; + DATA = 2; + TRANSACTION_META = 3; + SCHEMA_CHANGE = 4; + } + // The value of the Debezium message string payload = 1; string partition = 2; string offset = 3; string full_table_name = 4; int64 source_ts_ms = 5; - bool is_transaction_meta = 6; + CdcMessageType msg_type = 6; + // The key of the Debezium message, which only used by `mongodb-cdc` connector. string key = 7; } diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 1b4f4e423949e..f78c08e2a9b52 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -5,6 +5,7 @@ package ddl_service; import "catalog.proto"; import "common.proto"; import "meta.proto"; +import "plan_common.proto"; import "stream_plan.proto"; option java_package = "com.risingwave.proto"; @@ -444,6 +445,23 @@ message CommentOnResponse { uint64 version = 2; } +message TableSchemaChange { + enum TableChangeType { + UNSPECIFIED = 0; + ALTER = 1; + CREATE = 2; + DROP = 3; + } + + TableChangeType change_type = 1; + string cdc_table_name = 2; + repeated plan_common.ColumnCatalog columns = 3; +} + +message SchemaChangeEnvelope { + repeated TableSchemaChange table_changes = 1; +} + service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 642b363a8c67e..fa80069080ff4 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -301,6 +301,14 @@ impl<'a> JsonbRef<'a> { .ok_or_else(|| format!("cannot cast jsonb {} to type boolean", self.type_name())) } + /// If the JSON is a string, returns the associated string. + pub fn as_string(&self) -> Result { + self.0 + .as_str() + .map(|s| s.to_owned()) + .ok_or_else(|| format!("cannot cast jsonb {} to type string", self.type_name())) + } + /// Attempt to read jsonb as a JSON number. /// /// According to RFC 8259, only number within IEEE 754 binary64 (double precision) has good diff --git a/src/connector/codec/src/decoder/mod.rs b/src/connector/codec/src/decoder/mod.rs index cd7fe14ab74ea..814e06a166c6c 100644 --- a/src/connector/codec/src/decoder/mod.rs +++ b/src/connector/codec/src/decoder/mod.rs @@ -44,6 +44,9 @@ pub enum AccessError { #[error(transparent)] NotImplemented(#[from] NotImplemented), + // NOTE: We intentionally don't embed `anyhow::Error` in `AccessError` since it happens + // in record-level and it might be too heavy to capture the backtrace + // when creating a new `anyhow::Error`. } pub type AccessResult = std::result::Result; diff --git a/src/connector/src/parser/debezium/mod.rs b/src/connector/src/parser/debezium/mod.rs index 5b5416e647268..8852bfc25eb41 100644 --- a/src/connector/src/parser/debezium/mod.rs +++ b/src/connector/src/parser/debezium/mod.rs @@ -17,7 +17,9 @@ mod avro_parser; mod debezium_parser; mod mongo_json_parser; +pub mod schema_change; pub mod simd_json_parser; + pub use avro_parser::*; pub use debezium_parser::*; pub use mongo_json_parser::DebeziumMongoJsonParser; diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs new file mode 100644 index 0000000000000..4c61b52caaba9 --- /dev/null +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -0,0 +1,94 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_common::catalog::ColumnCatalog; +use risingwave_pb::ddl_service::table_schema_change::TableChangeType as PbTableChangeType; +use risingwave_pb::ddl_service::{ + SchemaChangeEnvelope as PbSchemaChangeEnvelope, TableSchemaChange as PbTableSchemaChange, +}; + +#[derive(Debug)] +pub struct SchemaChangeEnvelope { + pub table_changes: Vec, +} + +#[derive(Debug, Clone, Copy, PartialEq)] +pub(crate) enum TableChangeType { + Unspecified, + Alter, + Create, + Drop, +} + +impl TableChangeType { + #[allow(dead_code)] + pub fn from_proto(value: PbTableChangeType) -> Self { + match value { + PbTableChangeType::Alter => TableChangeType::Alter, + PbTableChangeType::Create => TableChangeType::Create, + PbTableChangeType::Drop => TableChangeType::Drop, + PbTableChangeType::Unspecified => TableChangeType::Unspecified, + } + } + + pub fn to_proto(self) -> PbTableChangeType { + match self { + TableChangeType::Alter => PbTableChangeType::Alter, + TableChangeType::Create => PbTableChangeType::Create, + TableChangeType::Drop => PbTableChangeType::Drop, + TableChangeType::Unspecified => PbTableChangeType::Unspecified, + } + } +} + +impl From<&str> for TableChangeType { + fn from(value: &str) -> Self { + match value { + "ALTER" => TableChangeType::Alter, + "CREATE" => TableChangeType::Create, + "DROP" => TableChangeType::Drop, + _ => TableChangeType::Unspecified, + } + } +} + +#[derive(Debug)] +pub struct TableSchemaChange { + pub(crate) cdc_table_name: String, + pub(crate) columns: Vec, + pub(crate) change_type: TableChangeType, +} + +impl SchemaChangeEnvelope { + pub fn to_protobuf(&self) -> PbSchemaChangeEnvelope { + let table_changes = self + .table_changes + .iter() + .map(|table_change| { + let columns = table_change + .columns + .iter() + .map(|column| column.to_protobuf()) + .collect(); + PbTableSchemaChange { + change_type: table_change.change_type.to_proto() as _, + cdc_table_name: table_change.cdc_table_name.clone(), + columns, + } + }) + .collect(); + + PbSchemaChangeEnvelope { table_changes } + } +} diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index f9738eb9e357e..08dd4ef7c2bdc 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -37,6 +37,13 @@ impl DebeziumJsonAccessBuilder { json_parse_options: JsonParseOptions::new_for_debezium(timestamptz_handling), }) } + + pub fn new_for_schema_event() -> ConnectorResult { + Ok(Self { + value: None, + json_parse_options: JsonParseOptions::default(), + }) + } } impl AccessBuilder for DebeziumJsonAccessBuilder { diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 25a7d8c169ab5..055eab777be5b 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -89,6 +89,7 @@ mod unified; mod upsert_parser; mod util; +use debezium::schema_change::SchemaChangeEnvelope; pub use debezium::DEBEZIUM_IGNORE_KEY; use risingwave_common::bitmap::BitmapBuilder; pub use unified::{AccessError, AccessResult}; @@ -579,6 +580,9 @@ pub enum ParseResult { Rows, /// A transaction control message is parsed. TransactionControl(TransactionControl), + + /// A schema change message is parsed. + SchemaChange(SchemaChangeEnvelope), } #[derive(Clone, Copy, Debug, PartialEq)] @@ -829,6 +833,10 @@ async fn into_chunk_stream_inner( } } }, + + Ok(ParseResult::SchemaChange(_)) => { + // TODO + } } } diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 1454ca8ade1fc..663fcb30e6ac9 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -23,10 +23,11 @@ use super::{ use crate::error::ConnectorResult; use crate::parser::bytes_parser::BytesAccessBuilder; use crate::parser::simd_json_parser::DebeziumJsonAccessBuilder; -use crate::parser::unified::debezium::parse_transaction_meta; +use crate::parser::unified::debezium::{parse_schema_change, parse_transaction_meta}; use crate::parser::unified::AccessImpl; use crate::parser::upsert_parser::get_key_column_name; use crate::parser::{BytesProperties, ParseResult, ParserFormat}; +use crate::source::cdc::CdcMessageType; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMeta}; /// Parser for `FORMAT PLAIN`, i.e., append-only source. @@ -38,6 +39,7 @@ pub struct PlainParser { pub source_ctx: SourceContextRef, // parsing transaction metadata for shared cdc source pub transaction_meta_builder: Option, + pub schema_change_builder: Option, } impl PlainParser { @@ -69,12 +71,18 @@ impl PlainParser { let transaction_meta_builder = Some(AccessBuilderImpl::DebeziumJson( DebeziumJsonAccessBuilder::new(TimestamptzHandling::GuessNumberUnit)?, )); + + let schema_change_builder = Some(AccessBuilderImpl::DebeziumJson( + DebeziumJsonAccessBuilder::new_for_schema_event()?, + )); + Ok(Self { key_builder, payload_builder, rw_columns, source_ctx, transaction_meta_builder, + schema_change_builder, }) } @@ -82,26 +90,62 @@ impl PlainParser { &mut self, key: Option>, payload: Option>, - mut writer: SourceStreamChunkRowWriter<'_>, + writer: SourceStreamChunkRowWriter<'_>, ) -> ConnectorResult { - // if the message is transaction metadata, parse it and return + // plain parser also used in the shared cdc source, + // we need to handle transaction metadata and schema change messages here if let Some(msg_meta) = writer.row_meta && let SourceMeta::DebeziumCdc(cdc_meta) = msg_meta.meta - && cdc_meta.is_transaction_meta && let Some(data) = payload { - let accessor = self - .transaction_meta_builder - .as_mut() - .expect("expect transaction metadata access builder") - .generate_accessor(data) - .await?; - return match parse_transaction_meta(&accessor, &self.source_ctx.connector_props) { - Ok(transaction_control) => Ok(ParseResult::TransactionControl(transaction_control)), - Err(err) => Err(err)?, - }; + match cdc_meta.msg_type { + CdcMessageType::Data | CdcMessageType::Heartbeat => { + return self.parse_rows(key, Some(data), writer).await; + } + CdcMessageType::TransactionMeta => { + let accessor = self + .transaction_meta_builder + .as_mut() + .expect("expect transaction metadata access builder") + .generate_accessor(data) + .await?; + return match parse_transaction_meta(&accessor, &self.source_ctx.connector_props) + { + Ok(transaction_control) => { + Ok(ParseResult::TransactionControl(transaction_control)) + } + Err(err) => Err(err)?, + }; + } + CdcMessageType::SchemaChange => { + let accessor = self + .schema_change_builder + .as_mut() + .expect("expect schema change access builder") + .generate_accessor(data) + .await?; + + return match parse_schema_change(&accessor, &self.source_ctx.connector_props) { + Ok(schema_change) => Ok(ParseResult::SchemaChange(schema_change)), + Err(err) => Err(err)?, + }; + } + CdcMessageType::Unspecified => { + unreachable!() + } + } } + // for non-cdc source messages + self.parse_rows(key, payload, writer).await + } + + async fn parse_rows( + &mut self, + key: Option>, + payload: Option>, + mut writer: SourceStreamChunkRowWriter<'_>, + ) -> ConnectorResult { let mut row_op: KvEvent, AccessImpl<'_>> = KvEvent::default(); if let Some(data) = key @@ -158,11 +202,13 @@ mod tests { use std::ops::Deref; use std::sync::Arc; + use expect_test::expect; use futures::executor::block_on; use futures::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; + use risingwave_pb::connector_service::cdc_message; use super::*; use crate::parser::{MessageMeta, SourceStreamChunkBuilder, TransactionControl}; @@ -281,7 +327,11 @@ mod tests { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( "orders".to_string(), 0, - transactional, + if transactional { + cdc_message::CdcMessageType::TransactionMeta + } else { + cdc_message::CdcMessageType::Data + }, )), split_id: SplitId::from("1001"), offset: "0".into(), @@ -295,7 +345,7 @@ mod tests { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( "orders".to_string(), 0, - false, + cdc_message::CdcMessageType::Data, )), split_id: SplitId::from("1001"), offset: "0".into(), @@ -309,7 +359,11 @@ mod tests { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( "orders".to_string(), 0, - transactional, + if transactional { + cdc_message::CdcMessageType::TransactionMeta + } else { + cdc_message::CdcMessageType::Data + }, )), split_id: SplitId::from("1001"), offset: "0".into(), @@ -355,7 +409,11 @@ mod tests { let begin_msg = r#"{"schema":null,"payload":{"status":"BEGIN","id":"3E11FA47-71CA-11E1-9E33-C80AA9429562:23","event_count":null,"data_collections":null,"ts_ms":1704269323180}}"#; let commit_msg = r#"{"schema":null,"payload":{"status":"END","id":"3E11FA47-71CA-11E1-9E33-C80AA9429562:23","event_count":11,"data_collections":[{"data_collection":"public.orders_tx","event_count":5},{"data_collection":"public.person","event_count":6}],"ts_ms":1704269323180}}"#; - let cdc_meta = SourceMeta::DebeziumCdc(DebeziumCdcMeta::new("orders".to_string(), 0, true)); + let cdc_meta = SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( + "orders".to_string(), + 0, + cdc_message::CdcMessageType::TransactionMeta, + )); let msg_meta = MessageMeta { meta: &cdc_meta, split_id: "1001", @@ -393,4 +451,120 @@ mod tests { let output = builder.take(10); assert_eq!(0, output.cardinality()); } + + #[tokio::test] + async fn test_parse_schema_change() { + let schema = vec![ + ColumnCatalog { + column_desc: ColumnDesc::named("payload", ColumnId::placeholder(), DataType::Jsonb), + is_hidden: false, + }, + ColumnCatalog::offset_column(), + ColumnCatalog::cdc_table_name_column(), + ]; + + let columns = schema + .iter() + .map(|c| SourceColumnDesc::from(&c.column_desc)) + .collect::>(); + + // format plain encode json parser + let source_ctx = SourceContext { + connector_props: ConnectorProperties::MysqlCdc(Box::default()), + ..SourceContext::dummy() + }; + let mut parser = PlainParser::new( + SpecificParserConfig::DEFAULT_PLAIN_JSON, + columns.clone(), + Arc::new(source_ctx), + ) + .await + .unwrap(); + let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 0); + + let msg = r#"{"schema":null,"payload": { "databaseName": "mydb", "ddl": "ALTER TABLE test add column v2 varchar(32)", "schemaName": null, "source": { "connector": "mysql", "db": "mydb", "file": "binlog.000065", "gtid": null, "name": "RW_CDC_1", "pos": 234, "query": null, "row": 0, "sequence": null, "server_id": 1, "snapshot": "false", "table": "test", "thread": null, "ts_ms": 1718354727000, "version": "2.4.2.Final" }, "tableChanges": [ { "id": "\"mydb\".\"test\"", "table": { "columns": [ { "autoIncremented": false, "charsetName": null, "comment": null, "defaultValueExpression": null, "enumValues": null, "generated": false, "jdbcType": 4, "length": null, "name": "id", "nativeType": null, "optional": false, "position": 1, "scale": null, "typeExpression": "INT", "typeName": "INT" }, { "autoIncremented": false, "charsetName": null, "comment": null, "defaultValueExpression": null, "enumValues": null, "generated": false, "jdbcType": 2014, "length": null, "name": "v1", "nativeType": null, "optional": true, "position": 2, "scale": null, "typeExpression": "TIMESTAMP", "typeName": "TIMESTAMP" }, { "autoIncremented": false, "charsetName": "utf8mb4", "comment": null, "defaultValueExpression": null, "enumValues": null, "generated": false, "jdbcType": 12, "length": 32, "name": "v2", "nativeType": null, "optional": true, "position": 3, "scale": null, "typeExpression": "VARCHAR", "typeName": "VARCHAR" } ], "comment": null, "defaultCharsetName": "utf8mb4", "primaryKeyColumnNames": [ "id" ] }, "type": "ALTER" } ], "ts_ms": 1718354727594 }}"#; + let cdc_meta = SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( + "mydb.test".to_string(), + 0, + cdc_message::CdcMessageType::SchemaChange, + )); + let msg_meta = MessageMeta { + meta: &cdc_meta, + split_id: "1001", + offset: "", + }; + + let res = parser + .parse_one_with_txn( + None, + Some(msg.as_bytes().to_vec()), + builder.row_writer().with_meta(msg_meta), + ) + .await; + + let res = res.unwrap(); + expect![[r#" + SchemaChange( + SchemaChangeEnvelope { + table_changes: [ + TableSchemaChange { + cdc_table_name: "mydb.test", + columns: [ + ColumnCatalog { + column_desc: ColumnDesc { + data_type: Int32, + column_id: #2147483646, + name: "id", + field_descs: [], + type_name: "", + generated_or_default_column: None, + description: None, + additional_column: AdditionalColumn { + column_type: None, + }, + version: Pr13707, + }, + is_hidden: false, + }, + ColumnCatalog { + column_desc: ColumnDesc { + data_type: Timestamptz, + column_id: #2147483646, + name: "v1", + field_descs: [], + type_name: "", + generated_or_default_column: None, + description: None, + additional_column: AdditionalColumn { + column_type: None, + }, + version: Pr13707, + }, + is_hidden: false, + }, + ColumnCatalog { + column_desc: ColumnDesc { + data_type: Varchar, + column_id: #2147483646, + name: "v2", + field_descs: [], + type_name: "", + generated_or_default_column: None, + description: None, + additional_column: AdditionalColumn { + column_type: None, + }, + version: Pr13707, + }, + is_hidden: false, + }, + ], + change_type: Alter, + }, + ], + }, + ) + "#]] + .assert_debug_eq(&res); + } } diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index d90463698577d..e4ec3f9870b43 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -12,14 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. +use itertools::Itertools; +use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; use risingwave_common::types::{ DataType, Datum, DatumCow, Scalar, ScalarImpl, ScalarRefImpl, Timestamptz, ToDatumRef, }; use risingwave_connector_codec::decoder::AccessExt; use risingwave_pb::plan_common::additional_column::ColumnType; +use thiserror_ext::AsReport; use super::{Access, AccessError, AccessResult, ChangeEvent, ChangeEventOperation}; +use crate::parser::debezium::schema_change::{SchemaChangeEnvelope, TableSchemaChange}; +use crate::parser::schema_change::TableChangeType; use crate::parser::TransactionControl; +use crate::source::cdc::external::mysql::{mysql_type_to_rw_type, type_name_to_mysql_type}; use crate::source::{ConnectorProperties, SourceColumnDesc}; // Example of Debezium JSON value: @@ -75,6 +81,8 @@ const OP: &str = "op"; pub const TRANSACTION_STATUS: &str = "status"; pub const TRANSACTION_ID: &str = "id"; +pub const TABLE_CHANGES: &str = "tableChanges"; + pub const DEBEZIUM_READ_OP: &str = "r"; pub const DEBEZIUM_CREATE_OP: &str = "c"; pub const DEBEZIUM_UPDATE_OP: &str = "u"; @@ -129,6 +137,96 @@ pub fn parse_transaction_meta( }) } +macro_rules! jsonb_access_field { + ($col:expr, $field:expr, $as_type:tt) => { + $crate::paste! { + $col.access_object_field($field).unwrap().[]().unwrap() + } + }; +} + +pub fn parse_schema_change( + accessor: &impl Access, + connector_props: &ConnectorProperties, +) -> AccessResult { + let mut schema_changes = vec![]; + + if let Some(ScalarRefImpl::List(table_changes)) = accessor + .access(&[TABLE_CHANGES], &DataType::List(Box::new(DataType::Jsonb)))? + .to_datum_ref() + { + for datum in table_changes.iter() { + let jsonb = match datum { + Some(ScalarRefImpl::Jsonb(jsonb)) => jsonb, + _ => unreachable!(""), + }; + + let id = jsonb_access_field!(jsonb, "id", string); + let ty = jsonb_access_field!(jsonb, "type", string); + let ddl_type: TableChangeType = ty.as_str().into(); + if matches!(ddl_type, TableChangeType::Create | TableChangeType::Drop) { + tracing::debug!("skip table schema change for create/drop command"); + continue; + } + + let mut column_descs: Vec = vec![]; + if let Some(table) = jsonb.access_object_field("table") + && let Some(columns) = table.access_object_field("columns") + { + for col in columns.array_elements().unwrap() { + let name = jsonb_access_field!(col, "name", string); + let type_name = jsonb_access_field!(col, "typeName", string); + + let data_type = match *connector_props { + ConnectorProperties::PostgresCdc(_) => { + unimplemented!() + } + ConnectorProperties::MysqlCdc(_) => { + let ty = type_name_to_mysql_type(type_name.as_str()); + match ty { + Some(ty) => mysql_type_to_rw_type(&ty).map_err(|err| { + tracing::warn!(error=%err.as_report(), "unsupported mysql type in schema change message"); + AccessError::UnsupportedType { + ty: type_name.clone(), + } + })?, + None => { + Err(AccessError::UnsupportedType { ty: type_name })? + } + } + } + _ => { + unreachable!() + } + }; + + column_descs.push(ColumnDesc::named(name, ColumnId::placeholder(), data_type)); + } + } + schema_changes.push(TableSchemaChange { + cdc_table_name: id.replace('"', ""), // remove the double quotes + columns: column_descs + .into_iter() + .map(|column_desc| ColumnCatalog { + column_desc, + is_hidden: false, + }) + .collect_vec(), + change_type: ty.as_str().into(), + }); + } + + Ok(SchemaChangeEnvelope { + table_changes: schema_changes, + }) + } else { + Err(AccessError::Undefined { + name: "table schema change".into(), + path: TABLE_CHANGES.into(), + }) + } +} + impl DebeziumChangeEvent where A: Access, diff --git a/src/connector/src/source/cdc/external/mysql.rs b/src/connector/src/source/cdc/external/mysql.rs index e5f53720dd6ee..6947ba7a46d6b 100644 --- a/src/connector/src/source/cdc/external/mysql.rs +++ b/src/connector/src/source/cdc/external/mysql.rs @@ -106,7 +106,7 @@ impl MySqlExternalTable { let mut column_descs = vec![]; let mut pk_names = vec![]; for col in columns { - let data_type = type_to_rw_type(&col.col_type)?; + let data_type = mysql_type_to_rw_type(&col.col_type)?; // column name in mysql is case-insensitive, convert to lowercase let col_name = col.name.to_lowercase(); column_descs.push(ColumnDesc::named( @@ -138,7 +138,56 @@ impl MySqlExternalTable { } } -fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { +pub fn type_name_to_mysql_type(ty_name: &str) -> Option { + macro_rules! column_type { + ($($name:literal => $variant:ident),* $(,)?) => { + match ty_name.to_lowercase().as_str() { + $( + $name => Some(ColumnType::$variant(Default::default())), + )* + _ => None, + } + }; + } + + column_type! { + "bit" => Bit, + "tinyint" => TinyInt, + "smallint" => SmallInt, + "mediumint" => MediumInt, + "int" => Int, + "bigint" => BigInt, + "decimal" => Decimal, + "float" => Float, + "double" => Double, + "time" => Time, + "datetime" => DateTime, + "timestamp" => Timestamp, + "char" => Char, + "nchar" => NChar, + "varchar" => Varchar, + "nvarchar" => NVarchar, + "binary" => Binary, + "varbinary" => Varbinary, + "text" => Text, + "tinytext" => TinyText, + "mediumtext" => MediumText, + "longtext" => LongText, + "blob" => Blob, + "enum" => Enum, + "set" => Set, + "geometry" => Geometry, + "point" => Point, + "linestring" => LineString, + "polygon" => Polygon, + "multipoint" => MultiPoint, + "multilinestring" => MultiLineString, + "multipolygon" => MultiPolygon, + "geometrycollection" => GeometryCollection, + } +} + +pub fn mysql_type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { let dtype = match col_type { ColumnType::Serial => DataType::Int32, ColumnType::Bit(attr) => { diff --git a/src/connector/src/source/cdc/source/message.rs b/src/connector/src/source/cdc/source/message.rs index f12d18339b527..831e242fe78de 100644 --- a/src/connector/src/source/cdc/source/message.rs +++ b/src/connector/src/source/cdc/source/message.rs @@ -13,11 +13,32 @@ // limitations under the License. use risingwave_common::types::{DatumRef, ScalarRefImpl, Timestamptz}; -use risingwave_pb::connector_service::CdcMessage; +use risingwave_pb::connector_service::{cdc_message, CdcMessage}; use crate::source::base::SourceMessage; use crate::source::SourceMeta; +#[derive(Clone, Debug)] +pub enum CdcMessageType { + Unspecified, + Heartbeat, + Data, + TransactionMeta, + SchemaChange, +} + +impl From for CdcMessageType { + fn from(msg_type: cdc_message::CdcMessageType) -> Self { + match msg_type { + cdc_message::CdcMessageType::Data => CdcMessageType::Data, + cdc_message::CdcMessageType::Heartbeat => CdcMessageType::Heartbeat, + cdc_message::CdcMessageType::TransactionMeta => CdcMessageType::TransactionMeta, + cdc_message::CdcMessageType::SchemaChange => CdcMessageType::SchemaChange, + cdc_message::CdcMessageType::Unspecified => CdcMessageType::Unspecified, + } + } +} + #[derive(Debug, Clone)] pub struct DebeziumCdcMeta { db_name_prefix_len: usize, @@ -25,11 +46,11 @@ pub struct DebeziumCdcMeta { pub full_table_name: String, // extracted from `payload.source.ts_ms`, the time that the change event was made in the database pub source_ts_ms: i64, - // Whether the message is a transaction metadata - pub is_transaction_meta: bool, + pub msg_type: CdcMessageType, } impl DebeziumCdcMeta { + // These `extract_xxx` methods are used to support the `INCLUDE TIMESTAMP/DATABASE_NAME/TABLE_NAME` feature pub fn extract_timestamp(&self) -> DatumRef<'_> { Some(ScalarRefImpl::Timestamptz( Timestamptz::from_millis(self.source_ts_ms).unwrap(), @@ -48,20 +69,25 @@ impl DebeziumCdcMeta { )) } - pub fn new(full_table_name: String, source_ts_ms: i64, is_transaction_meta: bool) -> Self { + pub fn new( + full_table_name: String, + source_ts_ms: i64, + msg_type: cdc_message::CdcMessageType, + ) -> Self { // full_table_name is in the format of `database_name.table_name` let db_name_prefix_len = full_table_name.as_str().find('.').unwrap_or(0); Self { db_name_prefix_len, full_table_name, source_ts_ms, - is_transaction_meta, + msg_type: msg_type.into(), } } } impl From for SourceMessage { fn from(message: CdcMessage) -> Self { + let msg_type = message.get_msg_type().expect("invalid message type"); SourceMessage { key: if message.key.is_empty() { None // only data message has key @@ -78,7 +104,7 @@ impl From for SourceMessage { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( message.full_table_name, message.source_ts_ms, - message.is_transaction_meta, + msg_type, )), } } From 3c745dfb008cdaad8b720c8787b1ff54f19195c8 Mon Sep 17 00:00:00 2001 From: xxchan Date: Sun, 4 Aug 2024 14:29:31 +0800 Subject: [PATCH 4/8] build: bump toolchain to 2024-06-12 (#17179) Signed-off-by: xxchan --- Cargo.lock | 24 ++- Cargo.toml | 11 +- ci/build-ci-image.sh | 2 +- ci/docker-compose.yml | 10 +- ci/rust-toolchain | 2 +- ci/scripts/build-other.sh | 2 +- clippy.toml | 3 + e2e_test/source_inline/pubsub/prepare-data.rs | 4 +- lints/Cargo.lock | 6 +- lints/Cargo.toml | 2 +- lints/README.md | 9 +- lints/rust-toolchain | 2 +- lints/src/format_error.rs | 14 +- lints/src/lib.rs | 13 +- lints/src/utils/format_args_collector.rs | 30 ++-- src/batch/src/executor/log_row_seq_scan.rs | 29 +-- src/batch/src/lib.rs | 1 - src/cmd_all/src/lib.rs | 1 - src/common/metrics/src/lib.rs | 1 - src/common/secret/src/lib.rs | 2 - src/common/src/lib.rs | 2 - src/compute/src/lib.rs | 1 - src/connector/benches/nexmark_integration.rs | 2 - src/connector/codec/src/lib.rs | 1 - src/connector/src/lib.rs | 1 - src/connector/src/parser/avro/parser.rs | 2 +- .../src/sink/formatter/append_only.rs | 29 +-- .../src/sink/formatter/debezium_json.rs | 167 +++++++++--------- src/connector/src/sink/formatter/upsert.rs | 37 ++-- src/connector/src/sink/kafka.rs | 17 +- src/connector/src/sink/kinesis.rs | 54 +++--- src/connector/src/sink/pulsar.rs | 21 ++- src/expr/core/src/lib.rs | 1 - src/expr/impl/src/lib.rs | 2 - src/expr/impl/src/window_function/buffer.rs | 2 +- src/frontend/src/lib.rs | 1 - src/jni_core/src/lib.rs | 30 ++-- src/jni_core/src/macros.rs | 1 - src/license/src/lib.rs | 2 - src/meta/service/src/lib.rs | 1 - src/meta/src/lib.rs | 1 - src/object_store/src/lib.rs | 1 - src/object_store/src/object/mod.rs | 5 + src/object_store/src/object/s3.rs | 5 +- src/storage/backup/src/lib.rs | 1 - src/storage/benches/bench_block_iter.rs | 1 - src/storage/benches/bench_table_watermarks.rs | 2 - src/storage/hummock_sdk/src/lib.rs | 1 - src/storage/hummock_trace/src/lib.rs | 1 - src/storage/src/lib.rs | 2 - src/storage/src/store.rs | 33 ++-- src/storage/src/store_impl.rs | 32 +++- src/stream/src/lib.rs | 1 - src/tests/simulation/src/lib.rs | 1 - src/tests/simulation/src/main.rs | 1 - .../tests/integration_tests/main.rs | 1 - .../integration_tests/scale/no_shuffle.rs | 4 +- src/tests/sqlsmith/src/lib.rs | 1 - src/utils/pgwire/src/lib.rs | 1 - 59 files changed, 344 insertions(+), 293 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f110fec5c16c4..220d030807e14 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5072,9 +5072,9 @@ dependencies = [ [[package]] name = "futures-async-stream" -version = "0.2.9" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "379790776b0d953337df4ab7ecc51936c66ea112484cad7912907b1d34253ebf" +checksum = "6cce57e88ba9fe4953f476112b2c8e315a2da07725a14dc091ac3e5b6e4cca72" dependencies = [ "futures-async-stream-macro", "futures-core", @@ -5083,9 +5083,9 @@ dependencies = [ [[package]] name = "futures-async-stream-macro" -version = "0.2.9" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5df2c13d48c8cb8a3ec093ede6f0f4482f327d7bb781120c5fb483ef0f17e758" +checksum = "5ac45ed0bddbd110eb68862768a194f88700f5b91c39931d2f432fab67a16d08" dependencies = [ "proc-macro2", "quote", @@ -7626,6 +7626,12 @@ dependencies = [ "num-traits", ] +[[package]] +name = "num-conv" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "51d515d32fb182ee37cda2ccdcb92950d6a3c2893aa280e540671c2cd0f3b1d9" + [[package]] name = "num-format" version = "0.4.4" @@ -13827,13 +13833,14 @@ dependencies = [ [[package]] name = "time" -version = "0.3.30" +version = "0.3.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4a34ab300f2dee6e562c10a046fc05e358b29f9bf92277f30c3c8d82275f6f5" +checksum = "5dfd88e563464686c916c7e46e623e520ddc6d79fa6641390f2e3fa86e83e885" dependencies = [ "deranged", "itoa", "libc", + "num-conv", "num_threads", "powerfmt", "serde", @@ -13849,10 +13856,11 @@ checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" [[package]] name = "time-macros" -version = "0.2.15" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ad70d68dba9e1f8aceda7aa6711965dfec1cac869f311a51bd08b3a2ccbce20" +checksum = "3f252a68540fde3a3877aeea552b832b40ab9a69e318efd078774a01ddee1ccf" dependencies = [ + "num-conv", "time-core", ] diff --git a/Cargo.toml b/Cargo.toml index 0c7b8a8999e60..1785398917b57 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -247,11 +247,16 @@ rw_iter_util = { path = "src/utils/iter_util" } [workspace.lints.rust] # `forbid` will also prevent the misuse of `#[allow(unused)]` unused_must_use = "forbid" -future_incompatible = "warn" -nonstandard_style = "warn" -rust_2018_idioms = "warn" +future_incompatible = { level = "warn", priority = -1 } +nonstandard_style = { level = "warn", priority = -1 } +rust_2018_idioms = { level = "warn", priority = -1 } # Backward compatibility is not important for an application. async_fn_in_trait = "allow" +unexpected_cfgs = { level = "warn", check-cfg = [ + 'cfg(madsim)', + 'cfg(coverage)', + 'cfg(dashboard_built)', +] } [workspace.lints.clippy] uninlined_format_args = "allow" diff --git a/ci/build-ci-image.sh b/ci/build-ci-image.sh index 329bf01b49c5b..6602509824e05 100755 --- a/ci/build-ci-image.sh +++ b/ci/build-ci-image.sh @@ -10,7 +10,7 @@ cat ../rust-toolchain # shellcheck disable=SC2155 # REMEMBER TO ALSO UPDATE ci/docker-compose.yml -export BUILD_ENV_VERSION=v20240729 +export BUILD_ENV_VERSION=v20240731 export BUILD_TAG="public.ecr.aws/w1p7b4n3/rw-build-env:${BUILD_ENV_VERSION}" diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index 89287926edc7f..83cb000566d4a 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -71,7 +71,7 @@ services: retries: 5 source-test-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240729 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240731 depends_on: - mysql - db @@ -84,7 +84,7 @@ services: - ..:/risingwave sink-test-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240729 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240731 depends_on: - mysql - db @@ -107,12 +107,12 @@ services: rw-build-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240729 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240731 volumes: - ..:/risingwave ci-flamegraph-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240729 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240731 # NOTE(kwannoel): This is used in order to permit # syscalls for `nperf` (perf_event_open), # so it can do CPU profiling. @@ -123,7 +123,7 @@ services: - ..:/risingwave regress-test-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240729 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240731 depends_on: db: condition: service_healthy diff --git a/ci/rust-toolchain b/ci/rust-toolchain index 92ef4a5ff27be..6bc57a2a65d8f 100644 --- a/ci/rust-toolchain +++ b/ci/rust-toolchain @@ -4,4 +4,4 @@ # 3. (optional) **follow the instructions in lints/README.md** to update the toolchain and dependencies for lints [toolchain] -channel = "nightly-2024-03-12" +channel = "nightly-2024-06-06" diff --git a/ci/scripts/build-other.sh b/ci/scripts/build-other.sh index 65c50462f97a0..d59e8ea876ccf 100755 --- a/ci/scripts/build-other.sh +++ b/ci/scripts/build-other.sh @@ -5,9 +5,9 @@ set -euo pipefail source ci/scripts/common.sh - echo "--- Build Rust UDF" cd e2e_test/udf/wasm +rustup target add wasm32-wasi cargo build --release cd ../../.. diff --git a/clippy.toml b/clippy.toml index 551de0eb6c479..21b972376b0ed 100644 --- a/clippy.toml +++ b/clippy.toml @@ -39,3 +39,6 @@ doc-valid-idents = [ avoid-breaking-exported-api = false upper-case-acronyms-aggressive = true too-many-arguments-threshold = 10 +ignore-interior-mutability = [ + "risingwave_frontend::expr::ExprImpl" # XXX: Where does ExprImpl have interior mutability? +] diff --git a/e2e_test/source_inline/pubsub/prepare-data.rs b/e2e_test/source_inline/pubsub/prepare-data.rs index a792c6cbb2618..e084b46919421 100755 --- a/e2e_test/source_inline/pubsub/prepare-data.rs +++ b/e2e_test/source_inline/pubsub/prepare-data.rs @@ -1,5 +1,5 @@ #!/usr/bin/env -S cargo -Zscript -```cargo +---cargo [dependencies] anyhow = "1" google-cloud-googleapis = { version = "0.13", features = ["pubsub"] } @@ -13,7 +13,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "signal", "fs", ] } -``` +--- use google_cloud_googleapis::pubsub::v1::PubsubMessage; use google_cloud_pubsub::client::{Client, ClientConfig}; diff --git a/lints/Cargo.lock b/lints/Cargo.lock index 3a9bd8384c0d9..e3b748e6da670 100644 --- a/lints/Cargo.lock +++ b/lints/Cargo.lock @@ -162,8 +162,7 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "clippy_config" -version = "0.1.79" -source = "git+https://github.com/rust-lang/rust-clippy?rev=fca4e16ffb8c07186ee23becd44cd5c9fb51896c#fca4e16ffb8c07186ee23becd44cd5c9fb51896c" +version = "0.1.80" dependencies = [ "rustc-semver", "serde", @@ -172,8 +171,7 @@ dependencies = [ [[package]] name = "clippy_utils" -version = "0.1.79" -source = "git+https://github.com/rust-lang/rust-clippy?rev=fca4e16ffb8c07186ee23becd44cd5c9fb51896c#fca4e16ffb8c07186ee23becd44cd5c9fb51896c" +version = "0.1.80" dependencies = [ "arrayvec", "clippy_config", diff --git a/lints/Cargo.toml b/lints/Cargo.toml index 74fc49c3fd080..43ece1f6fc5b7 100644 --- a/lints/Cargo.toml +++ b/lints/Cargo.toml @@ -14,7 +14,7 @@ path = "ui/format_error.rs" # See `README.md` before bumping the version. # Remember to update the version in `ci/Dockerfile` as well. [dependencies] -clippy_utils = { git = "https://github.com/rust-lang/rust-clippy", rev = "fca4e16ffb8c07186ee23becd44cd5c9fb51896c" } +clippy_utils = { git = "https://github.com/risingwavelabs/clippy", rev = "5e2a7c6adebdb0478ee6d5b67ab4ee94153b2997" } dylint_linting = "3.1.0" itertools = "0.12" diff --git a/lints/README.md b/lints/README.md index 5007474227ab3..3ab55f0bbfe7a 100644 --- a/lints/README.md +++ b/lints/README.md @@ -30,8 +30,13 @@ Duplicate `.vscode/settings.json.example` to `.vscode/settings.json` to enable r ## Bump toolchain -The version of the toolchain is specified in `rust-toolchain` file under current directory. -It does not have to be exactly the same as the one used to build RisingWave, but it should be close enough to avoid compile errors. +The version of the toolchain is specified in `rust-toolchain` file under current directory. It will be used to build the lints, and also be used by `dylint` to compile RisingWave, instead of the root-level `rust-toolchain`. + +So the chosen toolchain needs to +1. be close enough to the root-level `rust-toolchain` to make RisingWave compile. It does not have to be exactly the same version though. +2. be close enough to the dependency `clippy_utils`'s corresponding `rust-toolchain` in the Clippy's repo. + +(Note: `clippy_utils` depends on rustc's internal unstable API. When rustc has breaking changes, the `rust` repo's Clippy will be updated. And then it's [synced back to the Clippy repo bi-weekly](https://doc.rust-lang.org/clippy/development/infrastructure/sync.html#syncing-changes-between-clippy-and-rust-langrust). So ideally we can use `clippy_utils` in the rust repo corresponding to our root-level nightly version, but that repo is too large. Perhaps we can also consider copy the code out to workaround this problem.) The information below can be helpful in finding the appropriate version to bump to. diff --git a/lints/rust-toolchain b/lints/rust-toolchain index 3bbdf2b2d53fd..a146af66cd637 100644 --- a/lints/rust-toolchain +++ b/lints/rust-toolchain @@ -1,5 +1,5 @@ # See `README.md` before bumping the version. [toolchain] -channel = "nightly-2024-04-18" +channel = "nightly-2024-06-06" components = ["llvm-tools-preview", "rustc-dev"] diff --git a/lints/src/format_error.rs b/lints/src/format_error.rs index 9b6d9be5b8c4e..402adc4aa5af0 100644 --- a/lints/src/format_error.rs +++ b/lints/src/format_error.rs @@ -14,7 +14,7 @@ use clippy_utils::diagnostics::span_lint_and_help; use clippy_utils::macros::{ - find_format_arg_expr, find_format_args, is_format_macro, macro_backtrace, + find_format_arg_expr, is_format_macro, macro_backtrace, FormatArgsStorage, }; use clippy_utils::ty::{implements_trait, match_type}; use clippy_utils::{ @@ -56,7 +56,15 @@ declare_tool_lint! { } #[derive(Default)] -pub struct FormatError; +pub struct FormatError { + format_args: FormatArgsStorage, +} + +impl FormatError { + pub fn new(format_args: FormatArgsStorage) -> Self { + Self { format_args } + } +} impl_lint_pass!(FormatError => [FORMAT_ERROR]); @@ -90,7 +98,7 @@ impl<'tcx> LateLintPass<'tcx> for FormatError { for macro_call in macro_backtrace(expr.span) { if is_format_macro(cx, macro_call.def_id) - && let Some(format_args) = find_format_args(cx, expr, macro_call.expn) + && let Some(format_args) = self.format_args.get(cx, expr, macro_call.expn) { for piece in &format_args.template { if let FormatArgsPiece::Placeholder(placeholder) = piece diff --git a/lints/src/lib.rs b/lints/src/lib.rs index df77538d3cf17..6928bcd028a8c 100644 --- a/lints/src/lib.rs +++ b/lints/src/lib.rs @@ -14,7 +14,6 @@ #![feature(rustc_private)] #![feature(let_chains)] -#![feature(lazy_cell)] #![warn(unused_extern_crates)] extern crate rustc_ast; @@ -36,13 +35,19 @@ pub fn register_lints(_sess: &rustc_session::Session, lint_store: &mut rustc_lin // -- Begin lint registration -- // Preparation steps. - lint_store.register_early_pass(|| { - Box::::default() + let format_args_storage = clippy_utils::macros::FormatArgsStorage::default(); + let format_args = format_args_storage.clone(); + lint_store.register_early_pass(move || { + Box::new(utils::format_args_collector::FormatArgsCollector::new( + format_args.clone(), + )) }); // Actual lints. lint_store.register_lints(&[format_error::FORMAT_ERROR]); - lint_store.register_late_pass(|_| Box::::default()); + let format_args = format_args_storage.clone(); + lint_store + .register_late_pass(move |_| Box::new(format_error::FormatError::new(format_args.clone()))); // -- End lint registration -- diff --git a/lints/src/utils/format_args_collector.rs b/lints/src/utils/format_args_collector.rs index 7524169666d97..a3b144a6a8aeb 100644 --- a/lints/src/utils/format_args_collector.rs +++ b/lints/src/utils/format_args_collector.rs @@ -12,15 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! Copied from `https://github.com/rust-lang/rust-clippy/blob/8b0bf6423dfaf5545014db85fcba7bc745beed4c/clippy_lints/src/utils/format_args_collector.rs` -//! -//! Init `AST_FORMAT_ARGS` before running the late pass, so that we can call `find_format_args`. +//! Copied from `https://github.com/rust-lang/rust-clippy/blob/993d8ae2a7b26ac779fde923b2ce9ce35d7143a8/clippy_lints/src/utils/format_args_collector.rs` use std::iter::once; use std::mem; -use std::rc::Rc; -use clippy_utils::macros::AST_FORMAT_ARGS; +use clippy_utils::macros::FormatArgsStorage; use clippy_utils::source::snippet_opt; use itertools::Itertools; use rustc_ast::{Crate, Expr, ExprKind, FormatArgs}; @@ -30,11 +27,19 @@ use rustc_lint::{EarlyContext, EarlyLintPass}; use rustc_session::impl_lint_pass; use rustc_span::{hygiene, Span}; -/// Collects [`rustc_ast::FormatArgs`] so that future late passes can call -/// [`clippy_utils::macros::find_format_args`] -#[derive(Default)] +/// Populates [`FormatArgsStorage`] with AST [`FormatArgs`] nodes pub struct FormatArgsCollector { - format_args: FxHashMap>, + format_args: FxHashMap, + storage: FormatArgsStorage, +} + +impl FormatArgsCollector { + pub fn new(storage: FormatArgsStorage) -> Self { + Self { + format_args: FxHashMap::default(), + storage, + } + } } impl_lint_pass!(FormatArgsCollector => []); @@ -47,15 +52,12 @@ impl EarlyLintPass for FormatArgsCollector { } self.format_args - .insert(expr.span.with_parent(None), Rc::new((**args).clone())); + .insert(expr.span.with_parent(None), (**args).clone()); } } fn check_crate_post(&mut self, _: &EarlyContext<'_>, _: &Crate) { - AST_FORMAT_ARGS.with(|ast_format_args| { - let result = ast_format_args.set(mem::take(&mut self.format_args)); - debug_assert!(result.is_ok()); - }); + self.storage.set(mem::take(&mut self.format_args)); } } diff --git a/src/batch/src/executor/log_row_seq_scan.rs b/src/batch/src/executor/log_row_seq_scan.rs index 5783c788bb5e4..3614673ee941e 100644 --- a/src/batch/src/executor/log_row_seq_scan.rs +++ b/src/batch/src/executor/log_row_seq_scan.rs @@ -206,21 +206,24 @@ impl LogRowSeqScanExecutor { .batch_iter_log_with_pk_bounds(old_epoch, new_epoch) .await? .flat_map(|r| { - futures::stream::iter(std::iter::from_coroutine(move || { - match r { - Ok(change_log_row) => { - fn with_op(op: Op, row: impl Row) -> impl Row { - row.chain([Some(ScalarImpl::Int16(op.to_i16()))]) + futures::stream::iter(std::iter::from_coroutine( + #[coroutine] + move || { + match r { + Ok(change_log_row) => { + fn with_op(op: Op, row: impl Row) -> impl Row { + row.chain([Some(ScalarImpl::Int16(op.to_i16()))]) + } + for (op, row) in change_log_row.into_op_value_iter() { + yield Ok(with_op(op, row)); + } } - for (op, row) in change_log_row.into_op_value_iter() { - yield Ok(with_op(op, row)); + Err(e) => { + yield Err(e); } - } - Err(e) => { - yield Err(e); - } - }; - })) + }; + }, + )) }); pin_mut!(iter); diff --git a/src/batch/src/lib.rs b/src/batch/src/lib.rs index ce479daa2afc5..414f27b33b4a7 100644 --- a/src/batch/src/lib.rs +++ b/src/batch/src/lib.rs @@ -28,7 +28,6 @@ #![feature(allocator_api)] #![feature(impl_trait_in_assoc_type)] #![feature(assert_matches)] -#![feature(lazy_cell)] #![feature(error_generic_member_access)] #![feature(map_try_insert)] #![feature(iter_from_coroutine)] diff --git a/src/cmd_all/src/lib.rs b/src/cmd_all/src/lib.rs index 73180c75032ca..9b95af8dce4d3 100644 --- a/src/cmd_all/src/lib.rs +++ b/src/cmd_all/src/lib.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(lazy_cell)] #![feature(let_chains)] mod common; diff --git a/src/common/metrics/src/lib.rs b/src/common/metrics/src/lib.rs index fa0250f4f0c7b..35e8b11265843 100644 --- a/src/common/metrics/src/lib.rs +++ b/src/common/metrics/src/lib.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(lazy_cell)] #![feature(type_alias_impl_trait)] #![feature(impl_trait_in_assoc_type)] use std::ops::Deref; diff --git a/src/common/secret/src/lib.rs b/src/common/secret/src/lib.rs index 8ac065e5ea183..17319a296734d 100644 --- a/src/common/secret/src/lib.rs +++ b/src/common/secret/src/lib.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(lazy_cell)] - type SecretId = u32; mod secret_manager; diff --git a/src/common/src/lib.rs b/src/common/src/lib.rs index 467b313720f29..8d47d0c621646 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -26,7 +26,6 @@ #![feature(lint_reasons)] #![feature(coroutines)] #![feature(map_try_insert)] -#![feature(lazy_cell)] #![feature(error_generic_member_access)] #![feature(let_chains)] #![feature(portable_simd)] @@ -35,7 +34,6 @@ #![allow(incomplete_features)] #![feature(iterator_try_collect)] #![feature(iter_order_by)] -#![feature(exclusive_range_pattern)] #![feature(binary_heap_into_iter_sorted)] #![feature(impl_trait_in_assoc_type)] #![feature(map_entry_replace)] diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index cee364ceb26de..d91fb56b1cb88 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -18,7 +18,6 @@ #![feature(let_chains)] #![feature(lint_reasons)] #![feature(impl_trait_in_assoc_type)] -#![feature(lazy_cell)] #![cfg_attr(coverage, feature(coverage_attribute))] #[macro_use] diff --git a/src/connector/benches/nexmark_integration.rs b/src/connector/benches/nexmark_integration.rs index 172931562efef..f59ae08a9ec14 100644 --- a/src/connector/benches/nexmark_integration.rs +++ b/src/connector/benches/nexmark_integration.rs @@ -18,8 +18,6 @@ //! `ByteStreamSourceParserImpl::create` based on the given configuration, rather //! than depending on a specific internal implementation. -#![feature(lazy_cell)] - use std::sync::LazyLock; use criterion::{criterion_group, criterion_main, BatchSize, Criterion}; diff --git a/src/connector/codec/src/lib.rs b/src/connector/codec/src/lib.rs index 651fa84e109fb..cbf0ad14046f7 100644 --- a/src/connector/codec/src/lib.rs +++ b/src/connector/codec/src/lib.rs @@ -22,7 +22,6 @@ #![feature(box_patterns)] #![feature(trait_alias)] #![feature(lint_reasons)] -#![feature(lazy_cell)] #![feature(let_chains)] #![feature(box_into_inner)] #![feature(type_alias_impl_trait)] diff --git a/src/connector/src/lib.rs b/src/connector/src/lib.rs index 85bb7740ae9f9..02a3b8c84b50f 100644 --- a/src/connector/src/lib.rs +++ b/src/connector/src/lib.rs @@ -20,7 +20,6 @@ #![feature(box_patterns)] #![feature(trait_alias)] #![feature(lint_reasons)] -#![feature(lazy_cell)] #![feature(let_chains)] #![feature(box_into_inner)] #![feature(type_alias_impl_trait)] diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index dde74a999ac9f..ac93ab3e69807 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -87,7 +87,7 @@ impl AvroAccessBuilder { /// ## Confluent schema registry /// /// - In Kafka ([Confluent schema registry wire format](https://docs.confluent.io/platform/7.6/schema-registry/fundamentals/serdes-develop/index.html#wire-format)): - /// starts with 5 bytes`0x00{schema_id:08x}` followed by Avro binary encoding. + /// starts with 5 bytes`0x00{schema_id:08x}` followed by Avro binary encoding. async fn parse_avro_value(&self, payload: &[u8]) -> ConnectorResult> { // parse payload to avro value // if use confluent schema, get writer schema from confluent schema registry diff --git a/src/connector/src/sink/formatter/append_only.rs b/src/connector/src/sink/formatter/append_only.rs index 8a4b7feda6a42..c0c47fa37e171 100644 --- a/src/connector/src/sink/formatter/append_only.rs +++ b/src/connector/src/sink/formatter/append_only.rs @@ -40,19 +40,22 @@ impl SinkFormatter for AppendOnlyFormatter impl Iterator, Option)>> { - std::iter::from_coroutine(|| { - for (op, row) in chunk.rows() { - if op != Op::Insert { - continue; - } - let event_key_object = match &self.key_encoder { - Some(key_encoder) => Some(tri!(key_encoder.encode(row))), - None => None, - }; - let event_object = Some(tri!(self.val_encoder.encode(row))); + std::iter::from_coroutine( + #[coroutine] + || { + for (op, row) in chunk.rows() { + if op != Op::Insert { + continue; + } + let event_key_object = match &self.key_encoder { + Some(key_encoder) => Some(tri!(key_encoder.encode(row))), + None => None, + }; + let event_object = Some(tri!(self.val_encoder.encode(row))); - yield Ok((event_key_object, event_object)) - } - }) + yield Ok((event_key_object, event_object)) + } + }, + ) } } diff --git a/src/connector/src/sink/formatter/debezium_json.rs b/src/connector/src/sink/formatter/debezium_json.rs index fd4813e78541a..a9bf0404f473e 100644 --- a/src/connector/src/sink/formatter/debezium_json.rs +++ b/src/connector/src/sink/formatter/debezium_json.rs @@ -100,100 +100,103 @@ impl SinkFormatter for DebeziumJsonFormatter { &self, chunk: &StreamChunk, ) -> impl Iterator, Option)>> { - std::iter::from_coroutine(|| { - let DebeziumJsonFormatter { - schema, - pk_indices, - db_name, - sink_from_name, - opts, - key_encoder, - val_encoder, - } = self; - let ts_ms = SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_millis() as u64; - let source_field = json!({ - // todo: still some missing fields in source field - // ref https://debezium.io/documentation/reference/2.4/connectors/postgresql.html#postgresql-create-events - "db": db_name, - "table": sink_from_name, - "ts_ms": ts_ms, - }); - - let mut update_cache: Option> = None; - - for (op, row) in chunk.rows() { - let event_key_object: Option = Some(json!({ - "schema": json!({ - "type": "struct", - "fields": fields_pk_to_json(&schema.fields, pk_indices), - "optional": false, - "name": concat_debezium_name_field(db_name, sink_from_name, "Key"), - }), - "payload": tri!(key_encoder.encode(row)), - })); - let event_object: Option = match op { - Op::Insert => Some(json!({ - "schema": schema_to_json(schema, db_name, sink_from_name), - "payload": { - "before": null, - "after": tri!(val_encoder.encode(row)), - "op": "c", - "ts_ms": ts_ms, - "source": source_field, - } - })), - Op::Delete => { - let value_obj = Some(json!({ + std::iter::from_coroutine( + #[coroutine] + || { + let DebeziumJsonFormatter { + schema, + pk_indices, + db_name, + sink_from_name, + opts, + key_encoder, + val_encoder, + } = self; + let ts_ms = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() as u64; + let source_field = json!({ + // todo: still some missing fields in source field + // ref https://debezium.io/documentation/reference/2.4/connectors/postgresql.html#postgresql-create-events + "db": db_name, + "table": sink_from_name, + "ts_ms": ts_ms, + }); + + let mut update_cache: Option> = None; + + for (op, row) in chunk.rows() { + let event_key_object: Option = Some(json!({ + "schema": json!({ + "type": "struct", + "fields": fields_pk_to_json(&schema.fields, pk_indices), + "optional": false, + "name": concat_debezium_name_field(db_name, sink_from_name, "Key"), + }), + "payload": tri!(key_encoder.encode(row)), + })); + let event_object: Option = match op { + Op::Insert => Some(json!({ "schema": schema_to_json(schema, db_name, sink_from_name), "payload": { - "before": tri!(val_encoder.encode(row)), - "after": null, - "op": "d", + "before": null, + "after": tri!(val_encoder.encode(row)), + "op": "c", "ts_ms": ts_ms, "source": source_field, } - })); - yield Ok((event_key_object.clone(), value_obj)); - - if opts.gen_tombstone { - // Tomestone event - // https://debezium.io/documentation/reference/2.1/connectors/postgresql.html#postgresql-delete-events - yield Ok((event_key_object, None)); - } - - continue; - } - Op::UpdateDelete => { - update_cache = Some(tri!(val_encoder.encode(row))); - continue; - } - Op::UpdateInsert => { - if let Some(before) = update_cache.take() { - Some(json!({ + })), + Op::Delete => { + let value_obj = Some(json!({ "schema": schema_to_json(schema, db_name, sink_from_name), "payload": { - "before": before, - "after": tri!(val_encoder.encode(row)), - "op": "u", + "before": tri!(val_encoder.encode(row)), + "after": null, + "op": "d", "ts_ms": ts_ms, "source": source_field, } - })) - } else { - warn!( - "not found UpdateDelete in prev row, skipping, row index {:?}", - row.index() - ); + })); + yield Ok((event_key_object.clone(), value_obj)); + + if opts.gen_tombstone { + // Tomestone event + // https://debezium.io/documentation/reference/2.1/connectors/postgresql.html#postgresql-delete-events + yield Ok((event_key_object, None)); + } + continue; } - } - }; - yield Ok((event_key_object, event_object)); - } - }) + Op::UpdateDelete => { + update_cache = Some(tri!(val_encoder.encode(row))); + continue; + } + Op::UpdateInsert => { + if let Some(before) = update_cache.take() { + Some(json!({ + "schema": schema_to_json(schema, db_name, sink_from_name), + "payload": { + "before": before, + "after": tri!(val_encoder.encode(row)), + "op": "u", + "ts_ms": ts_ms, + "source": source_field, + } + })) + } else { + warn!( + "not found UpdateDelete in prev row, skipping, row index {:?}", + row.index() + ); + continue; + } + } + }; + yield Ok((event_key_object, event_object)); + } + }, + ) } } diff --git a/src/connector/src/sink/formatter/upsert.rs b/src/connector/src/sink/formatter/upsert.rs index 612c22aaaf86c..7e586a7917ea1 100644 --- a/src/connector/src/sink/formatter/upsert.rs +++ b/src/connector/src/sink/formatter/upsert.rs @@ -40,22 +40,25 @@ impl SinkFormatter for UpsertFormatter { &self, chunk: &StreamChunk, ) -> impl Iterator, Option)>> { - std::iter::from_coroutine(|| { - for (op, row) in chunk.rows() { - let event_key_object = Some(tri!(self.key_encoder.encode(row))); - - let event_object = match op { - Op::Insert | Op::UpdateInsert => Some(tri!(self.val_encoder.encode(row))), - // Empty value with a key - Op::Delete => None, - Op::UpdateDelete => { - // upsert semantic does not require update delete event - continue; - } - }; - - yield Ok((event_key_object, event_object)) - } - }) + std::iter::from_coroutine( + #[coroutine] + || { + for (op, row) in chunk.rows() { + let event_key_object = Some(tri!(self.key_encoder.encode(row))); + + let event_object = match op { + Op::Insert | Op::UpdateInsert => Some(tri!(self.val_encoder.encode(row))), + // Empty value with a key + Op::Delete => None, + Op::UpdateDelete => { + // upsert semantic does not require update delete event + continue; + } + }; + + yield Ok((event_key_object, event_object)) + } + }, + ) } } diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index c6e65fb00c399..3a6914d2249c5 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -403,7 +403,16 @@ struct KafkaPayloadWriter<'a> { config: &'a KafkaConfig, } -pub type KafkaSinkDeliveryFuture = impl TryFuture + Unpin + 'static; +mod opaque_type { + use super::*; + pub type KafkaSinkDeliveryFuture = impl TryFuture + Unpin + 'static; + + pub(super) fn map_delivery_future(future: DeliveryFuture) -> KafkaSinkDeliveryFuture { + future.map(KafkaPayloadWriter::<'static>::map_future_result) + } +} +use opaque_type::map_delivery_future; +pub use opaque_type::KafkaSinkDeliveryFuture; pub struct KafkaSinkWriter { formatter: SinkFormatterImpl, @@ -482,7 +491,7 @@ impl<'w> KafkaPayloadWriter<'w> { Ok(delivery_future) => { if self .add_future - .add_future_may_await(Self::map_delivery_future(delivery_future)) + .add_future_may_await(map_delivery_future(delivery_future)) .await? { tracing::warn!( @@ -567,10 +576,6 @@ impl<'w> KafkaPayloadWriter<'w> { Err(_) => Err(KafkaError::Canceled.into()), } } - - fn map_delivery_future(future: DeliveryFuture) -> KafkaSinkDeliveryFuture { - future.map(KafkaPayloadWriter::<'static>::map_future_result) - } } impl<'a> FormattedSink for KafkaPayloadWriter<'a> { diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index 771d3c8a6f91d..3b5d49da46037 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -201,8 +201,36 @@ impl KinesisSinkWriter { } } -pub type KinesisSinkPayloadWriterDeliveryFuture = - impl TryFuture + Unpin + Send + 'static; +mod opaque_type { + use super::*; + pub type KinesisSinkPayloadWriterDeliveryFuture = + impl TryFuture + Unpin + Send + 'static; + + impl KinesisSinkPayloadWriter { + pub(super) fn finish(self) -> KinesisSinkPayloadWriterDeliveryFuture { + async move { + let builder = self.builder.expect("should not be None"); + let context_fmt = format!( + "failed to put record to {}", + builder + .get_stream_name() + .as_ref() + .expect("should have set stream name") + ); + Retry::spawn( + ExponentialBackoff::from_millis(100).map(jitter).take(3), + || builder.clone().send(), + ) + .await + .with_context(|| context_fmt.clone()) + .map_err(SinkError::Kinesis)?; + Ok(()) + } + .boxed() + } + } +} +pub use opaque_type::KinesisSinkPayloadWriterDeliveryFuture; impl KinesisSinkPayloadWriter { fn put_record(&mut self, key: String, payload: Vec) { @@ -216,28 +244,6 @@ impl KinesisSinkPayloadWriter { ), ); } - - fn finish(self) -> KinesisSinkPayloadWriterDeliveryFuture { - async move { - let builder = self.builder.expect("should not be None"); - let context_fmt = format!( - "failed to put record to {}", - builder - .get_stream_name() - .as_ref() - .expect("should have set stream name") - ); - Retry::spawn( - ExponentialBackoff::from_millis(100).map(jitter).take(3), - || builder.clone().send(), - ) - .await - .with_context(|| context_fmt.clone()) - .map_err(SinkError::Kinesis)?; - Ok(()) - } - .boxed() - } } impl FormattedSink for KinesisSinkPayloadWriter { diff --git a/src/connector/src/sink/pulsar.rs b/src/connector/src/sink/pulsar.rs index 3f016ad94946d..a92d5b16f85e3 100644 --- a/src/connector/src/sink/pulsar.rs +++ b/src/connector/src/sink/pulsar.rs @@ -235,15 +235,20 @@ struct PulsarPayloadWriter<'w> { add_future: DeliveryFutureManagerAddFuture<'w, PulsarDeliveryFuture>, } -pub type PulsarDeliveryFuture = impl TryFuture + Unpin + 'static; - -fn may_delivery_future(future: SendFuture) -> PulsarDeliveryFuture { - future.map(|result| { - result - .map(|_| ()) - .map_err(|e: pulsar::Error| SinkError::Pulsar(anyhow!(e))) - }) +mod opaque_type { + use super::*; + pub type PulsarDeliveryFuture = impl TryFuture + Unpin + 'static; + + pub(super) fn may_delivery_future(future: SendFuture) -> PulsarDeliveryFuture { + future.map(|result| { + result + .map(|_| ()) + .map_err(|e: pulsar::Error| SinkError::Pulsar(anyhow!(e))) + }) + } } +use opaque_type::may_delivery_future; +pub use opaque_type::PulsarDeliveryFuture; impl PulsarSinkWriter { pub async fn new( diff --git a/src/expr/core/src/lib.rs b/src/expr/core/src/lib.rs index b250b8ce901f5..d45d4ca11f80a 100644 --- a/src/expr/core/src/lib.rs +++ b/src/expr/core/src/lib.rs @@ -15,7 +15,6 @@ #![feature(let_chains)] #![feature(lint_reasons)] #![feature(iterator_try_collect)] -#![feature(lazy_cell)] #![feature(coroutines)] #![feature(never_type)] #![feature(error_generic_member_access)] diff --git a/src/expr/impl/src/lib.rs b/src/expr/impl/src/lib.rs index 56bdbe3b81100..e5c69c2660eeb 100644 --- a/src/expr/impl/src/lib.rs +++ b/src/expr/impl/src/lib.rs @@ -25,8 +25,6 @@ #![feature(assert_matches)] #![feature(lint_reasons)] #![feature(iterator_try_collect)] -#![feature(exclusive_range_pattern)] -#![feature(lazy_cell)] #![feature(coroutines)] #![feature(test)] #![feature(iter_array_chunks)] diff --git a/src/expr/impl/src/window_function/buffer.rs b/src/expr/impl/src/window_function/buffer.rs index bd1c10d162b23..57217dda6fd4b 100644 --- a/src/expr/impl/src/window_function/buffer.rs +++ b/src/expr/impl/src/window_function/buffer.rs @@ -970,7 +970,7 @@ mod tests { let key = |key: i64| -> StateKey { StateKey { - order_key: memcmp_encoding::encode_value(&Some(ScalarImpl::from(key)), order_type) + order_key: memcmp_encoding::encode_value(Some(ScalarImpl::from(key)), order_type) .unwrap(), pk: OwnedRow::empty().into(), } diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 185d65cb567a0..4bdf9fa398f77 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -25,7 +25,6 @@ #![feature(assert_matches)] #![feature(lint_reasons)] #![feature(box_patterns)] -#![feature(lazy_cell)] #![feature(macro_metavar_expr)] #![feature(min_specialization)] #![feature(extend_one)] diff --git a/src/jni_core/src/lib.rs b/src/jni_core/src/lib.rs index 18d1807948d21..419f4ffd21cb5 100644 --- a/src/jni_core/src/lib.rs +++ b/src/jni_core/src/lib.rs @@ -13,7 +13,6 @@ // limitations under the License. #![feature(error_generic_member_access)] -#![feature(lazy_cell)] #![feature(once_cell_try)] #![feature(type_alias_impl_trait)] #![feature(try_blocks)] @@ -243,25 +242,28 @@ struct JavaClassMethodCache { utc: OnceLock, } -// TODO: may only return a RowRef -pub type StreamChunkRowIterator<'a> = impl Iterator + 'a; +mod opaque_type { + use super::*; + // TODO: may only return a RowRef + pub type StreamChunkRowIterator<'a> = impl Iterator + 'a; + + impl<'a> JavaBindingIteratorInner<'a> { + pub(super) fn from_chunk(chunk: &'a StreamChunk) -> JavaBindingIteratorInner<'a> { + JavaBindingIteratorInner::StreamChunk( + chunk + .rows() + .map(|(op, row)| (op.to_protobuf(), row.to_owned_row())), + ) + } + } +} +pub use opaque_type::StreamChunkRowIterator; pub type HummockJavaBindingIterator = BoxStream<'static, anyhow::Result<(Bytes, OwnedRow)>>; - pub enum JavaBindingIteratorInner<'a> { Hummock(HummockJavaBindingIterator), StreamChunk(StreamChunkRowIterator<'a>), } -impl<'a> JavaBindingIteratorInner<'a> { - fn from_chunk(chunk: &'a StreamChunk) -> JavaBindingIteratorInner<'a> { - JavaBindingIteratorInner::StreamChunk( - chunk - .rows() - .map(|(op, row)| (op.to_protobuf(), row.to_owned_row())), - ) - } -} - enum RowExtra { Op(Op), Key(Bytes), diff --git a/src/jni_core/src/macros.rs b/src/jni_core/src/macros.rs index 982ccda06ecf0..2e7d095e0bd45 100644 --- a/src/jni_core/src/macros.rs +++ b/src/jni_core/src/macros.rs @@ -375,7 +375,6 @@ macro_rules! to_jvalue { /// Generate the jni signature of a given function /// ``` -/// #![feature(lazy_cell)] /// use risingwave_jni_core::gen_jni_sig; /// assert_eq!(gen_jni_sig!(boolean f(int, short, byte[])), "(IS[B)Z"); /// assert_eq!( diff --git a/src/license/src/lib.rs b/src/license/src/lib.rs index 0e641be9789b1..7f2b25d8f3fb4 100644 --- a/src/license/src/lib.rs +++ b/src/license/src/lib.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(lazy_cell)] - mod feature; mod manager; diff --git a/src/meta/service/src/lib.rs b/src/meta/service/src/lib.rs index 80a83349f2cc7..9ab248802772e 100644 --- a/src/meta/service/src/lib.rs +++ b/src/meta/service/src/lib.rs @@ -14,7 +14,6 @@ #![feature(lint_reasons)] #![feature(let_chains)] -#![feature(lazy_cell)] #![feature(impl_trait_in_assoc_type)] #![cfg_attr(coverage, feature(coverage_attribute))] diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index 71c99a7e065b4..811b3b152d061 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -20,7 +20,6 @@ #![feature(extract_if)] #![feature(hash_extract_if)] #![feature(btree_extract_if)] -#![feature(lazy_cell)] #![feature(let_chains)] #![feature(error_generic_member_access)] #![feature(assert_matches)] diff --git a/src/object_store/src/lib.rs b/src/object_store/src/lib.rs index 811d482587acb..d9e768b7f0290 100644 --- a/src/object_store/src/lib.rs +++ b/src/object_store/src/lib.rs @@ -14,7 +14,6 @@ #![feature(trait_alias)] #![feature(type_alias_impl_trait)] -#![feature(lazy_cell)] #![feature(lint_reasons)] #![feature(error_generic_member_access)] #![feature(let_chains)] diff --git a/src/object_store/src/object/mod.rs b/src/object_store/src/object/mod.rs index 72ec08c46537b..8ee8dc078fe17 100644 --- a/src/object_store/src/object/mod.rs +++ b/src/object_store/src/object/mod.rs @@ -12,6 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![expect( + unexpected_cfgs, + reason = "feature(hdfs-backend) is banned https://github.com/risingwavelabs/risingwave/pull/7875" +)] + pub mod sim; use std::ops::{Range, RangeBounds}; use std::sync::Arc; diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 0ef12f3da3a3f..001eb8128a5b2 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -116,10 +116,7 @@ impl S3StreamingUploader { /// Reference: const MIN_PART_SIZE: usize = 5 * 1024 * 1024; const MAX_PART_SIZE: usize = 5 * 1024 * 1024 * 1024; - let part_size = config - .upload_part_size - .min(MAX_PART_SIZE) - .max(MIN_PART_SIZE); + let part_size = config.upload_part_size.clamp(MIN_PART_SIZE, MAX_PART_SIZE); Self { client, diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index 6b569277c54dd..dbb150030194e 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -21,7 +21,6 @@ #![feature(map_try_insert)] #![feature(hash_extract_if)] #![feature(btree_extract_if)] -#![feature(lazy_cell)] #![feature(let_chains)] #![feature(error_generic_member_access)] #![cfg_attr(coverage, feature(coverage_attribute))] diff --git a/src/storage/benches/bench_block_iter.rs b/src/storage/benches/bench_block_iter.rs index f58499e07282f..a0ea7cfd844d1 100644 --- a/src/storage/benches/bench_block_iter.rs +++ b/src/storage/benches/bench_block_iter.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(lazy_cell)] use std::sync::LazyLock; use bytes::{BufMut, Bytes, BytesMut}; diff --git a/src/storage/benches/bench_table_watermarks.rs b/src/storage/benches/bench_table_watermarks.rs index aeeee6927d36a..531652065f014 100644 --- a/src/storage/benches/bench_table_watermarks.rs +++ b/src/storage/benches/bench_table_watermarks.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(lazy_cell)] - use std::collections::hash_map::Entry; use std::collections::{HashMap, HashSet, VecDeque}; use std::sync::{Arc, LazyLock}; diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index 2ad827d8cae5f..c894ecf184128 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -22,7 +22,6 @@ #![feature(is_sorted)] #![feature(let_chains)] #![feature(btree_cursors)] -#![feature(lazy_cell)] mod key_cmp; use std::cmp::Ordering; diff --git a/src/storage/hummock_trace/src/lib.rs b/src/storage/hummock_trace/src/lib.rs index 64417832206e0..48b0a71010a74 100644 --- a/src/storage/hummock_trace/src/lib.rs +++ b/src/storage/hummock_trace/src/lib.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(lazy_cell)] #![feature(cursor_remaining)] #![feature(trait_alias)] #![feature(coroutines)] diff --git a/src/storage/src/lib.rs b/src/storage/src/lib.rs index ab69c45093b46..e11d3e1cee1ca 100644 --- a/src/storage/src/lib.rs +++ b/src/storage/src/lib.rs @@ -31,12 +31,10 @@ #![feature(is_sorted)] #![feature(btree_extract_if)] #![feature(exact_size_is_empty)] -#![feature(lazy_cell)] #![cfg_attr(coverage, feature(coverage_attribute))] #![recursion_limit = "256"] #![feature(error_generic_member_access)] #![feature(let_chains)] -#![feature(exclusive_range_pattern)] #![feature(impl_trait_in_assoc_type)] #![feature(maybe_uninit_uninit_array)] #![feature(maybe_uninit_array_assume_init)] diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index 15d60f6546f91..4b837ce6d098e 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -193,21 +193,24 @@ impl ChangeLogValue { } pub fn into_op_value_iter(self) -> impl Iterator { - std::iter::from_coroutine(move || match self { - Self::Insert(row) => { - yield (Op::Insert, row); - } - Self::Delete(row) => { - yield (Op::Delete, row); - } - Self::Update { - old_value, - new_value, - } => { - yield (Op::UpdateDelete, old_value); - yield (Op::UpdateInsert, new_value); - } - }) + std::iter::from_coroutine( + #[coroutine] + move || match self { + Self::Insert(row) => { + yield (Op::Insert, row); + } + Self::Delete(row) => { + yield (Op::Delete, row); + } + Self::Update { + old_value, + new_value, + } => { + yield (Op::UpdateDelete, old_value); + yield (Op::UpdateInsert, new_value); + } + }, + ) } } diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index 7e1c247586241..3ee9e849dda4c 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -42,9 +42,27 @@ use crate::monitor::{ use crate::opts::StorageOpts; use crate::StateStore; -pub type HummockStorageType = impl StateStore + AsHummock; -pub type MemoryStateStoreType = impl StateStore + AsHummock; -pub type SledStateStoreType = impl StateStore + AsHummock; +mod opaque_type { + use super::*; + + pub type HummockStorageType = impl StateStore + AsHummock; + pub type MemoryStateStoreType = impl StateStore + AsHummock; + pub type SledStateStoreType = impl StateStore + AsHummock; + + pub fn in_memory(state_store: MemoryStateStore) -> MemoryStateStoreType { + may_dynamic_dispatch(state_store) + } + + pub fn hummock(state_store: HummockStorage) -> HummockStorageType { + may_dynamic_dispatch(may_verify(state_store)) + } + + pub fn sled(state_store: SledStateStore) -> SledStateStoreType { + may_dynamic_dispatch(state_store) + } +} +use opaque_type::{hummock, in_memory, sled}; +pub use opaque_type::{HummockStorageType, MemoryStateStoreType, SledStateStoreType}; /// The type erased [`StateStore`]. #[derive(Clone, EnumAsInner)] @@ -114,7 +132,7 @@ impl StateStoreImpl { storage_metrics: Arc, ) -> Self { // The specific type of MemoryStateStoreType in deducted here. - Self::MemoryStateStore(may_dynamic_dispatch(state_store).monitored(storage_metrics)) + Self::MemoryStateStore(in_memory(state_store).monitored(storage_metrics)) } pub fn hummock( @@ -122,16 +140,14 @@ impl StateStoreImpl { storage_metrics: Arc, ) -> Self { // The specific type of HummockStateStoreType in deducted here. - Self::HummockStateStore( - may_dynamic_dispatch(may_verify(state_store)).monitored(storage_metrics), - ) + Self::HummockStateStore(hummock(state_store).monitored(storage_metrics)) } pub fn sled( state_store: SledStateStore, storage_metrics: Arc, ) -> Self { - Self::SledStateStore(may_dynamic_dispatch(state_store).monitored(storage_metrics)) + Self::SledStateStore(sled(state_store).monitored(storage_metrics)) } pub fn shared_in_memory_store(storage_metrics: Arc) -> Self { diff --git a/src/stream/src/lib.rs b/src/stream/src/lib.rs index a3fb15eef544a..876deabc80f98 100644 --- a/src/stream/src/lib.rs +++ b/src/stream/src/lib.rs @@ -29,7 +29,6 @@ #![feature(map_try_insert)] #![feature(never_type)] #![feature(btreemap_alloc)] -#![feature(lazy_cell)] #![feature(error_generic_member_access)] #![feature(btree_extract_if)] #![feature(iter_order_by)] diff --git a/src/tests/simulation/src/lib.rs b/src/tests/simulation/src/lib.rs index ae2614e094a6d..aa6303b8e2f65 100644 --- a/src/tests/simulation/src/lib.rs +++ b/src/tests/simulation/src/lib.rs @@ -14,7 +14,6 @@ #![feature(trait_alias)] #![feature(lint_reasons)] -#![feature(lazy_cell)] #![feature(let_chains)] #![feature(try_blocks)] #![feature(register_tool)] diff --git a/src/tests/simulation/src/main.rs b/src/tests/simulation/src/main.rs index 31cc488044648..e45c4a3285ac3 100644 --- a/src/tests/simulation/src/main.rs +++ b/src/tests/simulation/src/main.rs @@ -13,7 +13,6 @@ // limitations under the License. #![cfg_attr(not(madsim), allow(dead_code))] -#![feature(lazy_cell)] use std::path::PathBuf; diff --git a/src/tests/simulation/tests/integration_tests/main.rs b/src/tests/simulation/tests/integration_tests/main.rs index 0ad0725dde405..fe57a2c896f18 100644 --- a/src/tests/simulation/tests/integration_tests/main.rs +++ b/src/tests/simulation/tests/integration_tests/main.rs @@ -18,7 +18,6 @@ //! for the rationale behind this approach. #![feature(stmt_expr_attributes)] -#![feature(lazy_cell)] #![feature(extract_if)] mod backfill_tests; diff --git a/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs b/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs index 30d9e0c73fecd..e87659d4f54db 100644 --- a/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs +++ b/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs @@ -81,7 +81,9 @@ async fn test_delta_join() -> Result<()> { .assert_result_eq(result); #[allow(unused_assignments)] - test_times += 1; + { + test_times += 1; + } }; } diff --git a/src/tests/sqlsmith/src/lib.rs b/src/tests/sqlsmith/src/lib.rs index 0c215c9146a45..2eb9c59d1bf38 100644 --- a/src/tests/sqlsmith/src/lib.rs +++ b/src/tests/sqlsmith/src/lib.rs @@ -14,7 +14,6 @@ #![feature(let_chains)] #![feature(if_let_guard)] -#![feature(lazy_cell)] #![feature(box_patterns)] #![feature(register_tool)] #![register_tool(rw)] diff --git a/src/utils/pgwire/src/lib.rs b/src/utils/pgwire/src/lib.rs index 2279156d89b37..8d1c00541bb95 100644 --- a/src/utils/pgwire/src/lib.rs +++ b/src/utils/pgwire/src/lib.rs @@ -16,7 +16,6 @@ #![feature(trait_alias)] #![feature(iterator_try_collect)] #![feature(trusted_len)] -#![feature(lazy_cell)] #![feature(buf_read_has_data_left)] #![feature(round_char_boundary)] #![feature(never_type)] From bf1c1dba764989cc17b8c6972290de496702832a Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Mon, 5 Aug 2024 11:04:10 +0800 Subject: [PATCH 5/8] fix(meta): include secret in meta backup for etcd meta store (#17908) --- src/meta/src/backup_restore/meta_snapshot_builder.rs | 4 +++- src/meta/src/backup_restore/restore_impl/v1.rs | 1 + src/meta/src/model/mod.rs | 3 ++- src/storage/backup/src/meta_snapshot_v1.rs | 11 ++++++++++- 4 files changed, 16 insertions(+), 3 deletions(-) diff --git a/src/meta/src/backup_restore/meta_snapshot_builder.rs b/src/meta/src/backup_restore/meta_snapshot_builder.rs index bb8a1eb919fdc..6696bd534b78a 100644 --- a/src/meta/src/backup_restore/meta_snapshot_builder.rs +++ b/src/meta/src/backup_restore/meta_snapshot_builder.rs @@ -21,7 +21,7 @@ use risingwave_backup::meta_snapshot_v1::{ClusterMetadata, MetaSnapshotV1}; use risingwave_backup::MetaSnapshotId; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_pb::catalog::{ - Connection, Database, Function, Index, Schema, Sink, Source, Subscription, Table, View, + Connection, Database, Function, Index, Schema, Secret, Sink, Source, Subscription, Table, View, }; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::meta::SystemParams; @@ -124,6 +124,7 @@ impl MetaSnapshotV1Builder { .ok_or_else(|| anyhow!("cluster id not found in meta store"))? .into(); let subscription = Subscription::list_at_snapshot::(&meta_store_snapshot).await?; + let secret = Secret::list_at_snapshot::(&meta_store_snapshot).await?; self.snapshot.metadata = ClusterMetadata { default_cf, @@ -144,6 +145,7 @@ impl MetaSnapshotV1Builder { system_param, cluster_id, subscription, + secret, }; Ok(()) } diff --git a/src/meta/src/backup_restore/restore_impl/v1.rs b/src/meta/src/backup_restore/restore_impl/v1.rs index cc0fdb1baa410..0ceee8daa40ba 100644 --- a/src/meta/src/backup_restore/restore_impl/v1.rs +++ b/src/meta/src/backup_restore/restore_impl/v1.rs @@ -199,5 +199,6 @@ async fn restore_metadata( restore_system_param_model(&meta_store, &[snapshot.metadata.system_param]).await?; restore_cluster_id(&meta_store, snapshot.metadata.cluster_id.into()).await?; restore_metadata_model(&meta_store, &snapshot.metadata.subscription).await?; + restore_metadata_model(&meta_store, &snapshot.metadata.secret).await?; Ok(()) } diff --git a/src/meta/src/model/mod.rs b/src/meta/src/model/mod.rs index 6b5f629a7c04d..07f57ff1d6f6d 100644 --- a/src/meta/src/model/mod.rs +++ b/src/meta/src/model/mod.rs @@ -207,7 +207,8 @@ macro_rules! for_all_metadata_models { ($macro:ident) => { $macro! { // These items should be included in a meta snapshot. - // So be sure to update meta backup/restore when adding new items. + // Make sure to update the meta backup&restore methods accordingly when adding new items, + // referring to https://github.com/risingwavelabs/risingwave/pull/15371/commits/c5a75320845a38cfb43241ddee16fd5c0e47833b { risingwave_pb::hummock::HummockVersionStats }, { crate::hummock::model::CompactionGroup }, { risingwave_pb::catalog::Database }, diff --git a/src/storage/backup/src/meta_snapshot_v1.rs b/src/storage/backup/src/meta_snapshot_v1.rs index b66609ed4eb29..9b4145ea25632 100644 --- a/src/storage/backup/src/meta_snapshot_v1.rs +++ b/src/storage/backup/src/meta_snapshot_v1.rs @@ -20,7 +20,7 @@ use itertools::Itertools; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_pb::catalog::{ - Connection, Database, Function, Index, Schema, Sink, Source, Subscription, Table, View, + Connection, Database, Function, Index, Schema, Secret, Sink, Source, Subscription, Table, View, }; use risingwave_pb::hummock::{CompactionGroup, HummockVersionStats, PbHummockVersion}; use risingwave_pb::meta::{SystemParams, TableFragments}; @@ -72,6 +72,10 @@ impl Display for ClusterMetadata { writeln!(f, "{:#?}", self.system_param)?; writeln!(f, "cluster_id:")?; writeln!(f, "{:#?}", self.cluster_id)?; + writeln!(f, "subscription:")?; + writeln!(f, "{:#?}", self.subscription)?; + writeln!(f, "secret:")?; + writeln!(f, "{:#?}", self.secret)?; Ok(()) } } @@ -121,6 +125,7 @@ pub struct ClusterMetadata { pub system_param: SystemParams, pub cluster_id: String, pub subscription: Vec, + pub secret: Vec, } impl ClusterMetadata { @@ -146,6 +151,7 @@ impl ClusterMetadata { Self::encode_prost_message(&self.system_param, buf); Self::encode_prost_message(&self.cluster_id, buf); Self::encode_prost_message_list(&self.subscription.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.secret.iter().collect_vec(), buf); Ok(()) } @@ -175,6 +181,8 @@ impl ClusterMetadata { let cluster_id: String = Self::decode_prost_message(&mut buf)?; let subscription: Vec = Self::try_decode_prost_message_list(&mut buf).unwrap_or_else(|| Ok(vec![]))?; + let secret: Vec = + Self::try_decode_prost_message_list(&mut buf).unwrap_or_else(|| Ok(vec![]))?; Ok(Self { default_cf, @@ -195,6 +203,7 @@ impl ClusterMetadata { system_param, cluster_id, subscription, + secret, }) } From faf4ecd792a3df6da8ff94324d9ba743ccde8b62 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Mon, 5 Aug 2024 11:05:54 +0800 Subject: [PATCH 6/8] fix(sink): license SQL Server sink (#17910) --- src/connector/src/sink/sqlserver.rs | 4 ++++ src/license/src/feature.rs | 1 + 2 files changed, 5 insertions(+) diff --git a/src/connector/src/sink/sqlserver.rs b/src/connector/src/sink/sqlserver.rs index c90ba256533fc..0b8c7fa1f779e 100644 --- a/src/connector/src/sink/sqlserver.rs +++ b/src/connector/src/sink/sqlserver.rs @@ -144,6 +144,10 @@ impl Sink for SqlServerSink { const SINK_NAME: &'static str = SQLSERVER_SINK; async fn validate(&self) -> Result<()> { + risingwave_common::license::Feature::SqlServerSink + .check_available() + .map_err(|e| anyhow::anyhow!(e))?; + if !self.is_append_only && self.pk_indices.is_empty() { return Err(SinkError::Config(anyhow!( "Primary key not defined for upsert SQL Server sink (please define in `primary_key` field)"))); diff --git a/src/license/src/feature.rs b/src/license/src/feature.rs index 186e15c998ae4..9f59b63cc51cf 100644 --- a/src/license/src/feature.rs +++ b/src/license/src/feature.rs @@ -47,6 +47,7 @@ macro_rules! for_all_features { { TimeTravel, Paid, "Query historical data within the retention period."}, { GlueSchemaRegistry, Paid, "Use Schema Registry from AWS Glue rather than Confluent." }, { SecretManagement, Paid, "Secret management." }, + { SqlServerSink, Paid, "Sink data from RisingWave to SQL Server." }, } }; } From 4987ccb839f523b7542059879f06d4ce7355f823 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 5 Aug 2024 14:03:19 +0800 Subject: [PATCH 7/8] feat(cdc): add license check to auto schema map (#17903) --- .../cdc_inline/auto_schema_map_mysql.slt | 19 +++++++++++++++ src/frontend/src/handler/create_table.rs | 24 +++++++++++++------ src/license/src/feature.rs | 5 ++-- 3 files changed, 39 insertions(+), 9 deletions(-) diff --git a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt index eaa727b610828..b1a4cd0836349 100644 --- a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt +++ b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt @@ -40,6 +40,9 @@ mysql --protocol=tcp -u root mytest -e " INSERT INTO mysql_types_test VALUES ( True, 1, -128, -32767, -8388608, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, 'a', 'b', '', '', '1001-01-01', '00:00:00', '1998-01-01 00:00:00.000000', '1970-01-01 00:00:01', 'sad', '[3,4]'); " +statement ok +ALTER SYSTEM SET license_key TO ''; + statement ok create source mysql_source with ( connector = 'mysql-cdc', @@ -51,6 +54,22 @@ create source mysql_source with ( server.id = '5601' ); +statement error +create table rw_customers (*) from mysql_source table 'mytest.customers'; +---- +db error: ERROR: Failed to run the query + +Caused by: + Not supported: feature CdcTableSchemaMap is only available for tier Paid and above, while the current tier is Free + +Hint: You may want to set a license key with `ALTER SYSTEM SET license_key = '...';` command. +HINT: Please define the schema manually + + + +statement ok +ALTER SYSTEM SET license_key TO DEFAULT; + statement ok create table rw_customers (*) from mysql_source table 'mytest.customers'; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 5dce6028d0cc0..9855843ad0015 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -26,6 +26,7 @@ use risingwave_common::catalog::{ CdcTableDesc, ColumnCatalog, ColumnDesc, TableId, TableVersionId, DEFAULT_SCHEMA_NAME, INITIAL_TABLE_VERSION_ID, }; +use risingwave_common::license::Feature; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::util::value_encoding::DatumToProtoExt; use risingwave_connector::source::cdc::external::{ @@ -45,6 +46,7 @@ use risingwave_sqlparser::ast::{ ExplainOptions, Format, ObjectName, OnConflict, SourceWatermark, TableConstraint, }; use risingwave_sqlparser::parser::IncludeOption; +use thiserror_ext::AsReport; use super::RwPgResponse; use crate::binder::{bind_data_type, bind_struct_field, Clause}; @@ -185,7 +187,7 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> _ => { return Err(ErrorCode::NotSupported( format!("{} is not a collatable data type", data_type), - "The only built-in collatable data types are `varchar`, please check your type".into() + "The only built-in collatable data types are `varchar`, please check your type".into(), ).into()); } } @@ -332,12 +334,12 @@ pub fn bind_sql_column_constraints( if let Some(snapshot_value) = rewritten_expr_impl.try_fold_const() { let snapshot_value = snapshot_value?; - column_catalogs[idx].column_desc.generated_or_default_column = Some( - GeneratedOrDefaultColumn::DefaultColumn(DefaultColumnDesc { + column_catalogs[idx].column_desc.generated_or_default_column = + Some(GeneratedOrDefaultColumn::DefaultColumn(DefaultColumnDesc { snapshot_value: Some(snapshot_value.to_protobuf()), - expr: Some(expr_impl.to_expr_proto()), /* persist the original expression */ - }), - ); + expr: Some(expr_impl.to_expr_proto()), + // persist the original expression + })); } else { return Err(ErrorCode::BindError(format!( "Default expression used in column `{}` cannot be evaluated. \ @@ -1105,6 +1107,14 @@ async fn derive_schema_for_cdc_table( ) -> Result<(Vec, Vec)> { // read cdc table schema from external db or parsing the schema from SQL definitions if need_auto_schema_map { + Feature::CdcTableSchemaMap + .check_available() + .map_err(|err| { + ErrorCode::NotSupported( + err.to_report_string(), + "Please define the schema manually".to_owned(), + ) + })?; let (options, secret_refs) = connect_properties.into_parts(); let config = ExternalTableConfig::try_from_btreemap(options, secret_refs) .context("failed to extract external table config")?; @@ -1328,7 +1338,7 @@ pub async fn generate_stream_graph_for_table( .into(), "Remove the FORMAT and ENCODE specification".into(), ) - .into()) + .into()); } }; diff --git a/src/license/src/feature.rs b/src/license/src/feature.rs index 9f59b63cc51cf..d30bf71143b31 100644 --- a/src/license/src/feature.rs +++ b/src/license/src/feature.rs @@ -47,6 +47,7 @@ macro_rules! for_all_features { { TimeTravel, Paid, "Query historical data within the retention period."}, { GlueSchemaRegistry, Paid, "Use Schema Registry from AWS Glue rather than Confluent." }, { SecretManagement, Paid, "Secret management." }, + { CdcTableSchemaMap, Paid, "Automatically map upstream schema to CDC Table."}, { SqlServerSink, Paid, "Sink data from RisingWave to SQL Server." }, } }; @@ -84,9 +85,9 @@ for_all_features!(def_feature); #[derive(Debug, Error)] pub enum FeatureNotAvailable { #[error( - "feature {:?} is only available for tier {:?} and above, while the current tier is {:?}\n\n\ + "feature {:?} is only available for tier {:?} and above, while the current tier is {:?}\n\n\ Hint: You may want to set a license key with `ALTER SYSTEM SET license_key = '...';` command.", - feature, feature.min_tier(), current_tier, + feature, feature.min_tier(), current_tier, )] InsufficientTier { feature: Feature, From c4adffc647ad7085597d19c37e97d6ada16ca084 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Mon, 5 Aug 2024 14:26:12 +0800 Subject: [PATCH 8/8] refactor: add subscription via add mutation (#17897) --- proto/stream_plan.proto | 14 +-- src/compute/tests/cdc_tests.rs | 1 + src/meta/src/barrier/command.rs | 41 +++++-- src/meta/src/barrier/recovery.rs | 1 + src/stream/src/common/table/state_table.rs | 7 ++ src/stream/src/executor/chain.rs | 1 + src/stream/src/executor/mod.rs | 106 ++++++++++++------ src/stream/src/executor/mview/materialize.rs | 55 ++++----- .../src/executor/source/source_executor.rs | 2 + src/stream/src/executor/values.rs | 1 + 10 files changed, 143 insertions(+), 86 deletions(-) diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 51bb66289e60e..ef0d9f0b55879 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -28,6 +28,7 @@ message AddMutation { // We may embed a pause mutation here. // TODO: we may allow multiple mutations in a single barrier. bool pause = 4; + repeated SubscriptionUpstreamInfo subscriptions_to_add = 5; } message StopMutation { @@ -95,14 +96,13 @@ message CombinedMutation { repeated BarrierMutation mutations = 1; } -message CreateSubscriptionMutation { - uint32 subscription_id = 1; +message SubscriptionUpstreamInfo { + uint32 subscriber_id = 1; uint32 upstream_mv_table_id = 2; } -message DropSubscriptionMutation { - uint32 subscription_id = 1; - uint32 upstream_mv_table_id = 2; +message DropSubscriptionsMutation { + repeated SubscriptionUpstreamInfo info = 1; } message BarrierMutation { @@ -122,10 +122,8 @@ message BarrierMutation { ResumeMutation resume = 8; // Throttle specific source exec or chain exec. ThrottleMutation throttle = 10; - // Create subscription on mv - CreateSubscriptionMutation create_subscription = 11; // Drop subscription on mv - DropSubscriptionMutation drop_subscription = 12; + DropSubscriptionsMutation drop_subscriptions = 12; // Combined mutation. CombinedMutation combined = 100; } diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index ac48c42c22e92..aaaa6321666eb 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -311,6 +311,7 @@ async fn test_cdc_backfill() -> StreamResult<()> { added_actors: HashSet::new(), splits, pause: false, + subscriptions_to_add: vec![], })); tx.send_barrier(init_barrier); diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index c427ce892c333..234c4b36c1c5e 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::{HashMap, HashSet}; +use std::fmt::Formatter; use std::sync::Arc; use futures::future::try_join_all; @@ -33,9 +34,9 @@ use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::throttle_mutation::RateLimit; use risingwave_pb::stream_plan::update_mutation::*; use risingwave_pb::stream_plan::{ - AddMutation, BarrierMutation, CombinedMutation, CreateSubscriptionMutation, Dispatcher, - Dispatchers, DropSubscriptionMutation, PauseMutation, ResumeMutation, - SourceChangeSplitMutation, StopMutation, StreamActor, ThrottleMutation, UpdateMutation, + AddMutation, BarrierMutation, CombinedMutation, Dispatcher, Dispatchers, + DropSubscriptionsMutation, PauseMutation, ResumeMutation, SourceChangeSplitMutation, + StopMutation, StreamActor, SubscriptionUpstreamInfo, ThrottleMutation, UpdateMutation, }; use risingwave_pb::stream_service::WaitEpochCommitRequest; use thiserror_ext::AsReport; @@ -278,7 +279,7 @@ pub enum Command { retention_second: u64, }, - /// `DropSubscription` command generates a `DropSubscriptionMutation` to notify + /// `DropSubscription` command generates a `DropSubscriptionsMutation` to notify /// materialize executor to stop storing old value when there is no /// subscription depending on it. DropSubscription { @@ -443,6 +444,17 @@ pub struct CommandContext { pub _span: tracing::Span, } +impl std::fmt::Debug for CommandContext { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("CommandContext") + .field("prev_epoch", &self.prev_epoch.value().0) + .field("curr_epoch", &self.curr_epoch.value().0) + .field("kind", &self.kind) + .field("command", &self.command) + .finish() + } +} + impl CommandContext { #[allow(clippy::too_many_arguments)] pub(super) fn new( @@ -554,6 +566,7 @@ impl CommandContext { actor_splits, // If the cluster is already paused, the new actors should be paused too. pause: self.current_paused_reason.is_some(), + subscriptions_to_add: Default::default(), })); if let Some(ReplaceTablePlan { @@ -748,16 +761,24 @@ impl CommandContext { upstream_mv_table_id, subscription_id, .. - } => Some(Mutation::CreateSubscription(CreateSubscriptionMutation { - upstream_mv_table_id: upstream_mv_table_id.table_id, - subscription_id: *subscription_id, + } => Some(Mutation::Add(AddMutation { + actor_dispatchers: Default::default(), + added_actors: vec![], + actor_splits: Default::default(), + pause: false, + subscriptions_to_add: vec![SubscriptionUpstreamInfo { + upstream_mv_table_id: upstream_mv_table_id.table_id, + subscriber_id: *subscription_id, + }], })), Command::DropSubscription { upstream_mv_table_id, subscription_id, - } => Some(Mutation::DropSubscription(DropSubscriptionMutation { - upstream_mv_table_id: upstream_mv_table_id.table_id, - subscription_id: *subscription_id, + } => Some(Mutation::DropSubscriptions(DropSubscriptionsMutation { + info: vec![SubscriptionUpstreamInfo { + subscriber_id: *subscription_id, + upstream_mv_table_id: upstream_mv_table_id.table_id, + }], })), }; diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index e47bf3fc79dd3..6fbd89d149150 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -372,6 +372,7 @@ impl GlobalBarrierManager { added_actors: Default::default(), actor_splits: build_actor_connector_splits(&source_split_assignments), pause: paused_reason.is_some(), + subscriptions_to_add: Default::default(), }))); // Use a different `curr_epoch` for each recovery attempt. diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 6e4c083c33d4b..df85aa7f260ce 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -1157,6 +1157,13 @@ where op_consistency_level: StateTableOpConsistencyLevel, ) -> StreamExecutorResult<()> { if self.op_consistency_level != op_consistency_level { + info!( + ?new_epoch, + prev_op_consistency_level = ?self.op_consistency_level, + ?op_consistency_level, + table_id = self.table_id.table_id, + "switch to new op consistency level" + ); self.commit_inner(new_epoch, Some(op_consistency_level)) .await } else { diff --git a/src/stream/src/executor/chain.rs b/src/stream/src/executor/chain.rs index 8a4131bb2635c..6f198ff2b7e12 100644 --- a/src/stream/src/executor/chain.rs +++ b/src/stream/src/executor/chain.rs @@ -143,6 +143,7 @@ mod test { added_actors: maplit::hashset! { actor_id }, splits: Default::default(), pause: false, + subscriptions_to_add: vec![], }), )), Message::Chunk(StreamChunk::from_pretty("I\n + 3")), diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index 8d460e06f22c7..4bb45a0543532 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -40,10 +40,10 @@ use risingwave_pb::stream_plan::barrier_mutation::Mutation as PbMutation; use risingwave_pb::stream_plan::stream_message::StreamMessage; use risingwave_pb::stream_plan::update_mutation::{DispatcherUpdate, MergeUpdate}; use risingwave_pb::stream_plan::{ - BarrierMutation, CombinedMutation, CreateSubscriptionMutation, Dispatchers, - DropSubscriptionMutation, PauseMutation, PbAddMutation, PbBarrier, PbBarrierMutation, - PbDispatcher, PbStreamMessage, PbUpdateMutation, PbWatermark, ResumeMutation, - SourceChangeSplitMutation, StopMutation, ThrottleMutation, + BarrierMutation, CombinedMutation, Dispatchers, DropSubscriptionsMutation, PauseMutation, + PbAddMutation, PbBarrier, PbBarrierMutation, PbDispatcher, PbStreamMessage, PbUpdateMutation, + PbWatermark, ResumeMutation, SourceChangeSplitMutation, StopMutation, SubscriptionUpstreamInfo, + ThrottleMutation, }; use smallvec::SmallVec; @@ -274,6 +274,8 @@ pub struct AddMutation { // TODO: remove this and use `SourceChangesSplit` after we support multiple mutations. pub splits: HashMap>, pub pause: bool, + /// (`upstream_mv_table_id`, `subscriber_id`) + pub subscriptions_to_add: Vec<(TableId, u32)>, } /// See [`PbMutation`] for the semantics of each mutation. @@ -287,13 +289,9 @@ pub enum Mutation { Resume, Throttle(HashMap>), AddAndUpdate(AddMutation, UpdateMutation), - CreateSubscription { - subscription_id: u32, - upstream_mv_table_id: TableId, - }, - DropSubscription { - subscription_id: u32, - upstream_mv_table_id: TableId, + DropSubscriptions { + /// `subscriber` -> `upstream_mv_table_id` + subscriptions_to_drop: Vec<(u32, TableId)>, }, } @@ -446,8 +444,7 @@ impl Barrier { | Mutation::Resume | Mutation::SourceChangeSplit(_) | Mutation::Throttle(_) - | Mutation::CreateSubscription { .. } - | Mutation::DropSubscription { .. } => false, + | Mutation::DropSubscriptions { .. } => false, } } @@ -512,6 +509,31 @@ impl Barrier { pub fn tracing_context(&self) -> &TracingContext { &self.tracing_context } + + pub fn added_subscriber_on_mv_table( + &self, + mv_table_id: TableId, + ) -> impl Iterator + '_ { + if let Some(Mutation::Add(add)) | Some(Mutation::AddAndUpdate(add, _)) = + self.mutation.as_deref() + { + Some(add) + } else { + None + } + .into_iter() + .flat_map(move |add| { + add.subscriptions_to_add.iter().filter_map( + move |(upstream_mv_table_id, subscriber_id)| { + if *upstream_mv_table_id == mv_table_id { + Some(*subscriber_id) + } else { + None + } + }, + ) + }) + } } impl PartialEq for BarrierInner { @@ -581,6 +603,7 @@ impl Mutation { added_actors, splits, pause, + subscriptions_to_add, }) => PbMutation::Add(PbAddMutation { actor_dispatchers: adds .iter() @@ -596,6 +619,13 @@ impl Mutation { added_actors: added_actors.iter().copied().collect(), actor_splits: actor_splits_to_protobuf(splits), pause: *pause, + subscriptions_to_add: subscriptions_to_add + .iter() + .map(|(table_id, subscriber_id)| SubscriptionUpstreamInfo { + subscriber_id: *subscriber_id, + upstream_mv_table_id: table_id.table_id, + }) + .collect(), }), Mutation::SourceChangeSplit(changes) => PbMutation::Splits(SourceChangeSplitMutation { actor_splits: changes @@ -629,19 +659,18 @@ impl Mutation { }, ], }), - Mutation::CreateSubscription { - upstream_mv_table_id, - subscription_id, - } => PbMutation::CreateSubscription(CreateSubscriptionMutation { - upstream_mv_table_id: upstream_mv_table_id.table_id, - subscription_id: *subscription_id, - }), - Mutation::DropSubscription { - upstream_mv_table_id, - subscription_id, - } => PbMutation::DropSubscription(DropSubscriptionMutation { - upstream_mv_table_id: upstream_mv_table_id.table_id, - subscription_id: *subscription_id, + Mutation::DropSubscriptions { + subscriptions_to_drop, + } => PbMutation::DropSubscriptions(DropSubscriptionsMutation { + info: subscriptions_to_drop + .iter() + .map( + |(subscriber_id, upstream_mv_table_id)| SubscriptionUpstreamInfo { + subscriber_id: *subscriber_id, + upstream_mv_table_id: upstream_mv_table_id.table_id, + }, + ) + .collect(), }), } } @@ -712,6 +741,18 @@ impl Mutation { }) .collect(), pause: add.pause, + subscriptions_to_add: add + .subscriptions_to_add + .iter() + .map( + |SubscriptionUpstreamInfo { + subscriber_id, + upstream_mv_table_id, + }| { + (TableId::new(*upstream_mv_table_id), *subscriber_id) + }, + ) + .collect(), }), PbMutation::Splits(s) => { @@ -740,13 +781,12 @@ impl Mutation { .map(|(actor_id, limit)| (*actor_id, limit.rate_limit)) .collect(), ), - PbMutation::CreateSubscription(create) => Mutation::CreateSubscription { - upstream_mv_table_id: TableId::new(create.upstream_mv_table_id), - subscription_id: create.subscription_id, - }, - PbMutation::DropSubscription(drop) => Mutation::DropSubscription { - upstream_mv_table_id: TableId::new(drop.upstream_mv_table_id), - subscription_id: drop.subscription_id, + PbMutation::DropSubscriptions(drop) => Mutation::DropSubscriptions { + subscriptions_to_drop: drop + .info + .iter() + .map(|info| (info.subscriber_id, TableId::new(info.upstream_mv_table_id))) + .collect(), }, PbMutation::Combined(CombinedMutation { mutations }) => match &mutations[..] { [BarrierMutation { diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index f4340c2920481..9762d0ecd91ed 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -272,7 +272,7 @@ impl MaterializeExecutor { } Self::may_update_depended_subscriptions( &mut self.depended_subscription_ids, - b.mutation.as_deref(), + &b, mv_table_id, ); let op_consistency_level = get_op_consistency_level( @@ -305,51 +305,36 @@ impl MaterializeExecutor { /// return true when changed fn may_update_depended_subscriptions( depended_subscriptions: &mut HashSet, - mutation: Option<&Mutation>, + barrier: &Barrier, mv_table_id: TableId, ) { - let Some(mutation) = mutation else { - return; - }; - match mutation { - Mutation::CreateSubscription { - subscription_id, - upstream_mv_table_id, - } => { - if *upstream_mv_table_id == mv_table_id - && !depended_subscriptions.insert(*subscription_id) - { - warn!( - ?depended_subscriptions, - ?mv_table_id, - subscription_id, - "subscription id already exists" - ); - } + for subscriber_id in barrier.added_subscriber_on_mv_table(mv_table_id) { + if !depended_subscriptions.insert(subscriber_id) { + warn!( + ?depended_subscriptions, + ?mv_table_id, + subscriber_id, + "subscription id already exists" + ); } - Mutation::DropSubscription { - subscription_id, - upstream_mv_table_id, - } => { + } + + if let Some(Mutation::DropSubscriptions { + subscriptions_to_drop, + }) = barrier.mutation.as_deref() + { + for (subscriber_id, upstream_mv_table_id) in subscriptions_to_drop { if *upstream_mv_table_id == mv_table_id - && !depended_subscriptions.remove(subscription_id) + && !depended_subscriptions.remove(subscriber_id) { warn!( ?depended_subscriptions, ?mv_table_id, - subscription_id, - "drop non existing subscription id" + subscriber_id, + "drop non existing subscriber_id id" ); } } - Mutation::Stop(_) - | Mutation::Update(_) - | Mutation::Add(_) - | Mutation::SourceChangeSplit(_) - | Mutation::Pause - | Mutation::Resume - | Mutation::Throttle(_) - | Mutation::AddAndUpdate(_, _) => {} } } } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 8e5c3f9726c28..c550c0e12f030 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -909,6 +909,7 @@ mod tests { ], }, pause: false, + subscriptions_to_add: vec![], })); barrier_tx.send(init_barrier).unwrap(); @@ -998,6 +999,7 @@ mod tests { ], }, pause: false, + subscriptions_to_add: vec![], })); barrier_tx.send(init_barrier).unwrap(); diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index f2e737701cd21..83da0ff68a7d5 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -207,6 +207,7 @@ mod tests { added_actors: maplit::hashset! {actor_id}, splits: Default::default(), pause: false, + subscriptions_to_add: vec![], })); tx.send(first_message).unwrap();