From 7c31cf93c1286a53195e8a38ac48aaa9e3b98096 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Tue, 11 Jun 2024 11:27:24 +0800 Subject: [PATCH 01/37] feat(expr): add `jsonb_set` function (#17124) Signed-off-by: Runji Wang --- proto/expr.proto | 1 + src/expr/impl/src/scalar/jsonb_set.rs | 186 ++++++++++++++++++ src/expr/impl/src/scalar/mod.rs | 1 + src/frontend/src/binder/expr/function.rs | 1 + src/frontend/src/expr/pure.rs | 1 + .../src/optimizer/plan_expr_visitor/strong.rs | 1 + src/tests/regress/Cargo.toml | 1 + src/tests/regress/data/sql/jsonb.sql | 78 ++++---- 8 files changed, 231 insertions(+), 39 deletions(-) create mode 100644 src/expr/impl/src/scalar/jsonb_set.rs diff --git a/proto/expr.proto b/proto/expr.proto index 998ed63a4b084..9abb1d74f4955 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -280,6 +280,7 @@ message ExprNode { JSONB_PATH_QUERY_FIRST = 623; JSONB_POPULATE_RECORD = 629; JSONB_TO_RECORD = 630; + JSONB_SET = 631; // Non-pure functions below (> 1000) // ------------------------ diff --git a/src/expr/impl/src/scalar/jsonb_set.rs b/src/expr/impl/src/scalar/jsonb_set.rs new file mode 100644 index 0000000000000..e3efefb05d416 --- /dev/null +++ b/src/expr/impl/src/scalar/jsonb_set.rs @@ -0,0 +1,186 @@ +// 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 jsonbb::ValueRef; +use risingwave_common::types::{JsonbRef, JsonbVal, ListRef}; +use risingwave_expr::{function, ExprError, Result}; + +/// Returns `target` with the item designated by `path` replaced by `new_value`, or with `new_value` +/// added if `create_if_missing` is true (which is the default) and the item designated by path does +/// not exist. All earlier steps in the path must exist, or the `target` is returned unchanged. As +/// with the path oriented operators, negative integers that appear in the path count from the end +/// of JSON arrays. +/// +/// If the last path step is an array index that is out of range, and `create_if_missing` is true, +/// the new value is added at the beginning of the array if the index is negative, or at the end of +/// the array if it is positive. +/// +/// # Examples +/// +/// ```slt +/// query T +/// SELECT jsonb_set('[{"f1":1,"f2":null},2,null,3]', '{0,f1}', '[2,3,4]', false); +/// ---- +/// [{"f1": [2, 3, 4], "f2": null}, 2, null, 3] +/// +/// query T +/// SELECT jsonb_set('[{"f1":1,"f2":null},2]', '{0,f3}', '[2,3,4]'); +/// ---- +/// [{"f1": 1, "f2": null, "f3": [2, 3, 4]}, 2] +/// ``` +#[function("jsonb_set(jsonb, varchar[], jsonb, boolean) -> jsonb")] +fn jsonb_set4( + target: JsonbRef<'_>, + path: ListRef<'_>, + new_value: JsonbRef<'_>, + create_if_missing: bool, +) -> Result { + if target.is_scalar() { + return Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot set path in scalar".into(), + }); + } + let target: ValueRef<'_> = target.into(); + let new_value: ValueRef<'_> = new_value.into(); + let mut builder = jsonbb::Builder::>::with_capacity(target.capacity()); + jsonbb_set_path(target, path, 0, new_value, create_if_missing, &mut builder)?; + Ok(JsonbVal::from(builder.finish())) +} + +#[function("jsonb_set(jsonb, varchar[], jsonb) -> jsonb")] +fn jsonb_set3( + target: JsonbRef<'_>, + path: ListRef<'_>, + new_value: JsonbRef<'_>, +) -> Result { + jsonb_set4(target, path, new_value, true) +} + +/// Recursively set `path[i..]` in `target` to `new_value` and write the result to `builder`. +/// +/// Panics if `i` is out of bounds. +fn jsonbb_set_path( + target: ValueRef<'_>, + path: ListRef<'_>, + i: usize, + new_value: ValueRef<'_>, + create_if_missing: bool, + builder: &mut jsonbb::Builder, +) -> Result<()> { + let last_step = i == path.len() - 1; + match target { + ValueRef::Object(obj) => { + let key = path + .get(i) + .unwrap() + .ok_or_else(|| ExprError::InvalidParam { + name: "path", + reason: format!("path element at position {} is null", i + 1).into(), + })? + .into_utf8(); + builder.begin_object(); + for (k, v) in obj.iter() { + builder.add_string(k); + if k != key { + builder.add_value(v); + } else if last_step { + builder.add_value(new_value); + } else { + // recursively set path[i+1..] in v + jsonbb_set_path(v, path, i + 1, new_value, create_if_missing, builder)?; + } + } + if create_if_missing && last_step && !obj.contains_key(key) { + builder.add_string(key); + builder.add_value(new_value); + } + builder.end_object(); + Ok(()) + } + ValueRef::Array(array) => { + let key = path + .get(i) + .unwrap() + .ok_or_else(|| ExprError::InvalidParam { + name: "path", + reason: format!("path element at position {} is null", i + 1).into(), + })? + .into_utf8(); + let idx = key.parse::().map_err(|_| ExprError::InvalidParam { + name: "path", + reason: format!( + "path element at position {} is not an integer: \"{}\"", + i + 1, + key + ) + .into(), + })?; + let Some(idx) = normalize_array_index(array.len(), idx) else { + // out of bounds index + if create_if_missing { + builder.begin_array(); + // the new value is added at the beginning of the array if the index is negative + if idx < 0 { + builder.add_value(new_value); + } + for v in array.iter() { + builder.add_value(v); + } + // or at the end of the array if it is positive. + if idx >= 0 { + builder.add_value(new_value); + } + builder.end_array(); + } else { + builder.add_value(target); + } + return Ok(()); + }; + builder.begin_array(); + for (j, v) in array.iter().enumerate() { + if j != idx { + builder.add_value(v); + continue; + } + if last_step { + builder.add_value(new_value); + } else { + // recursively set path[i+1..] in v + jsonbb_set_path(v, path, i + 1, new_value, create_if_missing, builder)?; + } + } + builder.end_array(); + Ok(()) + } + _ => { + builder.add_value(target); + Ok(()) + } + } +} + +/// Normalizes an array index to `0..len`. +/// Negative indices count from the end. i.e. `-len..0 => 0..len`. +/// Returns `None` if index is out of bounds. +fn normalize_array_index(len: usize, index: i32) -> Option { + if index < -(len as i32) || index >= (len as i32) { + return None; + } + if index >= 0 { + Some(index as usize) + } else { + Some((len as i32 + index) as usize) + } +} diff --git a/src/expr/impl/src/scalar/mod.rs b/src/expr/impl/src/scalar/mod.rs index edbaaf4de01ab..c4e7990de133b 100644 --- a/src/expr/impl/src/scalar/mod.rs +++ b/src/expr/impl/src/scalar/mod.rs @@ -58,6 +58,7 @@ mod jsonb_info; mod jsonb_object; mod jsonb_path; mod jsonb_record; +mod jsonb_set; mod length; mod lower; mod make_time; diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index e1d8602c24878..74162c35e9820 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -1160,6 +1160,7 @@ impl Binder { ("jsonb_path_exists", raw_call(ExprType::JsonbPathExists)), ("jsonb_path_query_array", raw_call(ExprType::JsonbPathQueryArray)), ("jsonb_path_query_first", raw_call(ExprType::JsonbPathQueryFirst)), + ("jsonb_set", raw_call(ExprType::JsonbSet)), // Functions that return a constant value ("pi", pi()), // greatest and least diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 0199c86caeb3d..d03b7507fdfb1 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -209,6 +209,7 @@ impl ExprVisitor for ImpureAnalyzer { | Type::JsonbPathMatch | Type::JsonbPathQueryArray | Type::JsonbPathQueryFirst + | Type::JsonbSet | Type::IsJson | Type::ToJsonb | Type::Sind diff --git a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs index ea55085f5a07a..fefdd1e4547fc 100644 --- a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs +++ b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs @@ -289,6 +289,7 @@ impl Strong { | ExprType::JsonbPathQueryFirst | ExprType::JsonbPopulateRecord | ExprType::JsonbToRecord + | ExprType::JsonbSet | ExprType::Vnode | ExprType::Proctime | ExprType::PgSleep diff --git a/src/tests/regress/Cargo.toml b/src/tests/regress/Cargo.toml index 8c5478d13d52e..65248877adf60 100644 --- a/src/tests/regress/Cargo.toml +++ b/src/tests/regress/Cargo.toml @@ -26,6 +26,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "time", "signal", "process", + "io-util", ] } tracing = "0.1" tracing-subscriber = "0.3.17" diff --git a/src/tests/regress/data/sql/jsonb.sql b/src/tests/regress/data/sql/jsonb.sql index b8d1e24654af2..71ce107ef270c 100644 --- a/src/tests/regress/data/sql/jsonb.sql +++ b/src/tests/regress/data/sql/jsonb.sql @@ -1114,18 +1114,18 @@ select '{"a":1 , "b":2, "c":3}'::jsonb - '{b}'::text[]; select '{"a":1 , "b":2, "c":3}'::jsonb - '{c,b}'::text[]; select '{"a":1 , "b":2, "c":3}'::jsonb - '{}'::text[]; ---@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{n}', '[1,2,3]'); ---@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '[1,2,3]'); ---@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{d,1,0}', '[1,2,3]'); ---@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{d,NULL,0}', '[1,2,3]'); ---@ ---@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{n}', '{"1": 2}'); ---@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '{"1": 2}'); ---@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{d,1,0}', '{"1": 2}'); ---@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{d,NULL,0}', '{"1": 2}'); ---@ ---@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '"test"'); ---@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '{"f": "test"}'); +select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{n}', '[1,2,3]'); +select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '[1,2,3]'); +select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{d,1,0}', '[1,2,3]'); +select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{d,NULL,0}', '[1,2,3]'); + +select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{n}', '{"1": 2}'); +select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '{"1": 2}'); +select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{d,1,0}', '{"1": 2}'); +select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{d,NULL,0}', '{"1": 2}'); + +select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '"test"'); +select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '{"f": "test"}'); select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{n}'); select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{b,-1}'); @@ -1148,33 +1148,33 @@ select '[]'::jsonb - 1; select '"a"'::jsonb #- '{a}'; -- error select '{}'::jsonb #- '{a}'; select '[]'::jsonb #- '{a}'; ---@ select jsonb_set('"a"','{a}','"b"'); --error ---@ select jsonb_set('{}','{a}','"b"', false); ---@ select jsonb_set('[]','{1}','"b"', false); ---@ select jsonb_set('[{"f1":1,"f2":null},2,null,3]', '{0}','[2,3,4]', false); ---@ ---@ -- jsonb_set adding instead of replacing ---@ ---@ -- prepend to array ---@ select jsonb_set('{"a":1,"b":[0,1,2],"c":{"d":4}}','{b,-33}','{"foo":123}'); ---@ -- append to array ---@ select jsonb_set('{"a":1,"b":[0,1,2],"c":{"d":4}}','{b,33}','{"foo":123}'); ---@ -- check nesting levels addition ---@ select jsonb_set('{"a":1,"b":[4,5,[0,1,2],6,7],"c":{"d":4}}','{b,2,33}','{"foo":123}'); ---@ -- add new key ---@ select jsonb_set('{"a":1,"b":[0,1,2],"c":{"d":4}}','{c,e}','{"foo":123}'); ---@ -- adding doesn't do anything if elements before last aren't present ---@ select jsonb_set('{"a":1,"b":[0,1,2],"c":{"d":4}}','{x,-33}','{"foo":123}'); ---@ select jsonb_set('{"a":1,"b":[0,1,2],"c":{"d":4}}','{x,y}','{"foo":123}'); ---@ -- add to empty object ---@ select jsonb_set('{}','{x}','{"foo":123}'); ---@ --add to empty array ---@ select jsonb_set('[]','{0}','{"foo":123}'); ---@ select jsonb_set('[]','{99}','{"foo":123}'); ---@ select jsonb_set('[]','{-99}','{"foo":123}'); ---@ select jsonb_set('{"a": [1, 2, 3]}', '{a, non_integer}', '"new_value"'); ---@ select jsonb_set('{"a": {"b": [1, 2, 3]}}', '{a, b, non_integer}', '"new_value"'); ---@ select jsonb_set('{"a": {"b": [1, 2, 3]}}', '{a, b, NULL}', '"new_value"'); +select jsonb_set('"a"','{a}','"b"'); --error +select jsonb_set('{}','{a}','"b"', false); +select jsonb_set('[]','{1}','"b"', false); +select jsonb_set('[{"f1":1,"f2":null},2,null,3]', '{0}','[2,3,4]', false); + +-- jsonb_set adding instead of replacing + +-- prepend to array +select jsonb_set('{"a":1,"b":[0,1,2],"c":{"d":4}}','{b,-33}','{"foo":123}'); +-- append to array +select jsonb_set('{"a":1,"b":[0,1,2],"c":{"d":4}}','{b,33}','{"foo":123}'); +-- check nesting levels addition +select jsonb_set('{"a":1,"b":[4,5,[0,1,2],6,7],"c":{"d":4}}','{b,2,33}','{"foo":123}'); +-- add new key +select jsonb_set('{"a":1,"b":[0,1,2],"c":{"d":4}}','{c,e}','{"foo":123}'); +-- adding doesn't do anything if elements before last aren't present +select jsonb_set('{"a":1,"b":[0,1,2],"c":{"d":4}}','{x,-33}','{"foo":123}'); +select jsonb_set('{"a":1,"b":[0,1,2],"c":{"d":4}}','{x,y}','{"foo":123}'); +-- add to empty object +select jsonb_set('{}','{x}','{"foo":123}'); +--add to empty array +select jsonb_set('[]','{0}','{"foo":123}'); +select jsonb_set('[]','{99}','{"foo":123}'); +select jsonb_set('[]','{-99}','{"foo":123}'); +select jsonb_set('{"a": [1, 2, 3]}', '{a, non_integer}', '"new_value"'); +select jsonb_set('{"a": {"b": [1, 2, 3]}}', '{a, b, non_integer}', '"new_value"'); +select jsonb_set('{"a": {"b": [1, 2, 3]}}', '{a, b, NULL}', '"new_value"'); -- jsonb_set_lax From f568dfaa35877e2b5745703cb544d2823a9758b5 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 11 Jun 2024 13:14:39 +0800 Subject: [PATCH 02/37] perf(connector): avoid allocation for metadata column when parsing (#17169) --- src/connector/src/parser/mod.rs | 63 ++++++++++--------- src/connector/src/parser/util.rs | 18 +++--- .../src/source/cdc/source/message.rs | 27 ++++---- .../src/source/kafka/source/message.rs | 61 ++++++++++-------- 4 files changed, 86 insertions(+), 83 deletions(-) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index d439da2491e6b..f7667a66a3747 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -30,7 +30,7 @@ use risingwave_common::bail; use risingwave_common::catalog::{KAFKA_TIMESTAMP_COLUMN_NAME, TABLE_NAME_COLUMN_NAME}; use risingwave_common::log::LogSuppresser; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; -use risingwave_common::types::{Datum, DatumCow, Scalar, ScalarImpl}; +use risingwave_common::types::{Datum, DatumCow, DatumRef, ScalarRefImpl}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::tracing::InstrumentStream; use risingwave_connector_codec::decoder::avro::MapHandling; @@ -203,17 +203,17 @@ pub struct MessageMeta<'a> { offset: &'a str, } -impl MessageMeta<'_> { +impl<'a> MessageMeta<'a> { /// Extract the value for the given column. /// /// Returns `None` if the column is not a meta column. - fn value_for_column(self, desc: &SourceColumnDesc) -> Option { - match desc.column_type { + fn value_for_column(self, desc: &SourceColumnDesc) -> Option> { + let datum: DatumRef<'_> = match desc.column_type { // Row id columns are filled with `NULL` here and will be filled with the real // row id generated by `RowIdGenExecutor` later. - SourceColumnType::RowId => Datum::None.into(), + SourceColumnType::RowId => None, // Extract the offset from the meta data. - SourceColumnType::Offset => Datum::Some(self.offset.into()).into(), + SourceColumnType::Offset => Some(self.offset.into()), // Extract custom meta data per connector. SourceColumnType::Meta if let SourceMeta::Kafka(kafka_meta) = self.meta => { assert_eq!( @@ -221,14 +221,11 @@ impl MessageMeta<'_> { KAFKA_TIMESTAMP_COLUMN_NAME, "unexpected kafka meta column name" ); - kafka_meta - .timestamp - .map(|ts| { - risingwave_common::cast::i64_to_timestamptz(ts) - .unwrap() - .to_scalar_value() - }) - .into() + kafka_meta.timestamp.map(|ts| { + risingwave_common::cast::i64_to_timestamptz(ts) + .unwrap() + .into() + }) } SourceColumnType::Meta if let SourceMeta::DebeziumCdc(cdc_meta) = self.meta => { assert_eq!( @@ -236,19 +233,21 @@ impl MessageMeta<'_> { TABLE_NAME_COLUMN_NAME, "unexpected cdc meta column name" ); - Datum::Some(cdc_meta.full_table_name.as_str().into()).into() + Some(cdc_meta.full_table_name.as_str().into()) } // For other cases, return `None`. - SourceColumnType::Meta | SourceColumnType::Normal => None, - } + SourceColumnType::Meta | SourceColumnType::Normal => return None, + }; + + Some(datum) } } trait OpAction { type Output<'a>; - fn output_for<'a>(datum: Datum) -> Self::Output<'a>; + fn output_for<'a>(datum: impl Into>) -> Self::Output<'a>; fn apply(builder: &mut ArrayBuilderImpl, output: Self::Output<'_>); @@ -263,7 +262,7 @@ impl OpAction for OpActionInsert { type Output<'a> = DatumCow<'a>; #[inline(always)] - fn output_for<'a>(datum: Datum) -> Self::Output<'a> { + fn output_for<'a>(datum: impl Into>) -> Self::Output<'a> { datum.into() } @@ -289,7 +288,7 @@ impl OpAction for OpActionDelete { type Output<'a> = DatumCow<'a>; #[inline(always)] - fn output_for<'a>(datum: Datum) -> Self::Output<'a> { + fn output_for<'a>(datum: impl Into>) -> Self::Output<'a> { datum.into() } @@ -315,8 +314,9 @@ impl OpAction for OpActionUpdate { type Output<'a> = (DatumCow<'a>, DatumCow<'a>); #[inline(always)] - fn output_for<'a>(datum: Datum) -> Self::Output<'a> { - (datum.clone().into(), datum.into()) + fn output_for<'a>(datum: impl Into>) -> Self::Output<'a> { + let datum = datum.into(); + (datum.clone(), datum) } #[inline(always)] @@ -345,7 +345,7 @@ impl SourceStreamChunkRowWriter<'_> { } fn do_action<'a, A: OpAction>( - &mut self, + &'a mut self, mut f: impl FnMut(&SourceColumnDesc) -> AccessResult>, ) -> AccessResult<()> { let mut parse_field = |desc: &SourceColumnDesc| { @@ -411,10 +411,11 @@ impl SourceStreamChunkRowWriter<'_> { match self.row_meta { Some(row_meta) => { if let SourceMeta::DebeziumCdc(cdc_meta) = row_meta.meta { - Ok(A::output_for( - extract_cdc_meta_column(cdc_meta, col, desc.name.as_str())? - .unwrap_or(None), - )) + Ok(A::output_for(extract_cdc_meta_column( + cdc_meta, + col, + desc.name.as_str(), + )?)) } else { Err(AccessError::Uncategorized { message: "CDC metadata not found in the message".to_string(), @@ -439,7 +440,7 @@ impl SourceStreamChunkRowWriter<'_> { return Ok(A::output_for( self.row_meta .as_ref() - .map(|ele| ScalarImpl::Utf8(ele.split_id.to_string().into())), + .map(|ele| ScalarRefImpl::Utf8(ele.split_id)), )); } (_, &Some(AdditionalColumnType::Offset(_))) => { @@ -447,7 +448,7 @@ impl SourceStreamChunkRowWriter<'_> { return Ok(A::output_for( self.row_meta .as_ref() - .map(|ele| ScalarImpl::Utf8(ele.offset.to_string().into())), + .map(|ele| ScalarRefImpl::Utf8(ele.offset)), )); } (_, &Some(AdditionalColumnType::HeaderInner(ref header_inner))) => { @@ -461,7 +462,7 @@ impl SourceStreamChunkRowWriter<'_> { header_inner.data_type.as_ref(), ) }) - .unwrap_or(None), + .unwrap_or(Datum::None.into()), )) } (_, &Some(AdditionalColumnType::Headers(_))) => { @@ -477,7 +478,7 @@ impl SourceStreamChunkRowWriter<'_> { return Ok(A::output_for( self.row_meta .as_ref() - .map(|ele| ScalarImpl::Utf8(ele.split_id.to_string().into())), + .map(|ele| ScalarRefImpl::Utf8(ele.split_id)), )); } (_, _) => { diff --git a/src/connector/src/parser/util.rs b/src/connector/src/parser/util.rs index 4afbbb718d54a..30cb1fbf7d62e 100644 --- a/src/connector/src/parser/util.rs +++ b/src/connector/src/parser/util.rs @@ -17,7 +17,7 @@ use anyhow::Context; use bytes::Bytes; use reqwest::Url; use risingwave_common::bail; -use risingwave_common::types::Datum; +use risingwave_common::types::{Datum, DatumCow, DatumRef}; use risingwave_pb::data::DataType as PbDataType; use crate::aws_utils::load_file_descriptor_from_s3; @@ -132,19 +132,19 @@ pub(super) async fn bytes_from_url( } } -pub fn extreact_timestamp_from_meta(meta: &SourceMeta) -> Option { +pub fn extreact_timestamp_from_meta(meta: &SourceMeta) -> Option> { match meta { SourceMeta::Kafka(kafka_meta) => kafka_meta.extract_timestamp(), - SourceMeta::DebeziumCdc(cdc_meta) => cdc_meta.extract_timestamp(), + SourceMeta::DebeziumCdc(cdc_meta) => Some(cdc_meta.extract_timestamp()), _ => None, } } -pub fn extract_cdc_meta_column( - cdc_meta: &DebeziumCdcMeta, +pub fn extract_cdc_meta_column<'a>( + cdc_meta: &'a DebeziumCdcMeta, column_type: &additional_column::ColumnType, column_name: &str, -) -> AccessResult> { +) -> AccessResult> { match column_type { ColumnType::Timestamp(_) => Ok(cdc_meta.extract_timestamp()), ColumnType::DatabaseName(_) => Ok(cdc_meta.extract_database_name()), @@ -162,11 +162,11 @@ pub fn extract_headers_from_meta(meta: &SourceMeta) -> Option { } } -pub fn extract_header_inner_from_meta( - meta: &SourceMeta, +pub fn extract_header_inner_from_meta<'a>( + meta: &'a SourceMeta, inner_field: &str, data_type: Option<&PbDataType>, -) -> Option { +) -> Option> { match meta { SourceMeta::Kafka(kafka_meta) => kafka_meta.extract_header_inner(inner_field, data_type), /* expect output of type `bytea` or `varchar` */ _ => None, diff --git a/src/connector/src/source/cdc/source/message.rs b/src/connector/src/source/cdc/source/message.rs index e74ed55ce1f9f..f12d18339b527 100644 --- a/src/connector/src/source/cdc/source/message.rs +++ b/src/connector/src/source/cdc/source/message.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::{Datum, Scalar, ScalarImpl, Timestamptz}; +use risingwave_common::types::{DatumRef, ScalarRefImpl, Timestamptz}; use risingwave_pb::connector_service::CdcMessage; use crate::source::base::SourceMessage; @@ -30,27 +30,22 @@ pub struct DebeziumCdcMeta { } impl DebeziumCdcMeta { - pub fn extract_timestamp(&self) -> Option { - Some( - Timestamptz::from_millis(self.source_ts_ms) - .unwrap() - .to_scalar_value(), - ) - .into() + pub fn extract_timestamp(&self) -> DatumRef<'_> { + Some(ScalarRefImpl::Timestamptz( + Timestamptz::from_millis(self.source_ts_ms).unwrap(), + )) } - pub fn extract_database_name(&self) -> Option { - Some(ScalarImpl::from( - self.full_table_name.as_str()[0..self.db_name_prefix_len].to_string(), + pub fn extract_database_name(&self) -> DatumRef<'_> { + Some(ScalarRefImpl::Utf8( + &self.full_table_name.as_str()[0..self.db_name_prefix_len], )) - .into() } - pub fn extract_table_name(&self) -> Option { - Some(ScalarImpl::from( - self.full_table_name.as_str()[self.db_name_prefix_len..].to_string(), + pub fn extract_table_name(&self) -> DatumRef<'_> { + Some(ScalarRefImpl::Utf8( + &self.full_table_name.as_str()[self.db_name_prefix_len..], )) - .into() } pub fn new(full_table_name: String, source_ts_ms: i64, is_transaction_meta: bool) -> Self { diff --git a/src/connector/src/source/kafka/source/message.rs b/src/connector/src/source/kafka/source/message.rs index 0ef55dc79132d..247166a156763 100644 --- a/src/connector/src/source/kafka/source/message.rs +++ b/src/connector/src/source/kafka/source/message.rs @@ -12,10 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::borrow::Cow; + use itertools::Itertools; use rdkafka::message::{BorrowedMessage, Headers, OwnedHeaders}; use rdkafka::Message; -use risingwave_common::types::{Datum, ListValue, Scalar, ScalarImpl, StructValue}; +use risingwave_common::types::{ + Datum, DatumCow, DatumRef, ListValue, ScalarImpl, ScalarRefImpl, StructValue, +}; use risingwave_pb::data::data_type::TypeName as PbTypeName; use risingwave_pb::data::DataType as PbDataType; @@ -31,39 +35,42 @@ pub struct KafkaMeta { } impl KafkaMeta { - pub fn extract_timestamp(&self) -> Option { - self.timestamp - .map(|ts| { - risingwave_common::cast::i64_to_timestamptz(ts) - .unwrap() - .to_scalar_value() - }) - .into() + pub fn extract_timestamp(&self) -> Option> { + self.timestamp.map(|ts| { + Some(ScalarRefImpl::Timestamptz( + risingwave_common::cast::i64_to_timestamptz(ts).unwrap(), + )) + }) } - pub fn extract_header_inner( - &self, + pub fn extract_header_inner<'a>( + &'a self, inner_field: &str, data_type: Option<&PbDataType>, - ) -> Option { - let target_value = self - .headers - .as_ref() - .iter() - .find_map(|headers| { - headers - .iter() - .find(|header| header.key == inner_field) - .map(|header| header.value) - }) - .unwrap_or(None); // if not found the specified column, return None - if let Some(data_type) = data_type + ) -> Option> { + let target_value = self.headers.as_ref().iter().find_map(|headers| { + headers + .iter() + .find(|header| header.key == inner_field) + .map(|header| header.value) + })?; // if not found the specified column, return None + + let Some(target_value) = target_value else { + return Some(Datum::None.into()); + }; + + let datum = if let Some(data_type) = data_type && data_type.type_name == PbTypeName::Varchar as i32 { - Some(target_value.map(|byte| ScalarImpl::Utf8(String::from_utf8_lossy(byte).into()))) + match String::from_utf8_lossy(target_value) { + Cow::Borrowed(str) => Some(ScalarRefImpl::Utf8(str)).into(), + Cow::Owned(string) => Some(ScalarImpl::Utf8(string.into())).into(), + } } else { - Some(target_value.map(|byte| ScalarImpl::Bytea(byte.into()))) - } + Some(ScalarRefImpl::Bytea(target_value)).into() + }; + + Some(datum) } pub fn extract_headers(&self) -> Option { From 142478850dab8adf979e74d94d4dd1e9741895cc Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 11 Jun 2024 13:18:29 +0800 Subject: [PATCH 03/37] test(connector): add benchmark for parsing combined (struct type) nexmark events (#17189) Signed-off-by: Bugen Zhao --- src/connector/benches/nexmark_integration.rs | 78 ++++++++++++++------ 1 file changed, 54 insertions(+), 24 deletions(-) diff --git a/src/connector/benches/nexmark_integration.rs b/src/connector/benches/nexmark_integration.rs index c2c735d1c2089..172931562efef 100644 --- a/src/connector/benches/nexmark_integration.rs +++ b/src/connector/benches/nexmark_integration.rs @@ -26,7 +26,7 @@ use criterion::{criterion_group, criterion_main, BatchSize, Criterion}; use futures::{FutureExt, StreamExt, TryStreamExt}; use itertools::Itertools; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::ColumnId; +use risingwave_common::catalog::{ColumnDesc, ColumnId}; use risingwave_common::types::DataType; use risingwave_connector::parser::{ ByteStreamSourceParserImpl, CommonParserConfig, ParserConfig, SpecificParserConfig, @@ -37,15 +37,12 @@ use risingwave_connector::source::{ use tracing::Level; use tracing_subscriber::prelude::*; -static BATCH: LazyLock> = LazyLock::new(make_batch); +static BATCH: LazyLock> = LazyLock::new(|| make_batch(false)); +static STRUCT_BATCH: LazyLock> = LazyLock::new(|| make_batch(true)); -fn make_batch() -> Vec { +fn make_batch(use_struct: bool) -> Vec { let mut generator = nexmark::EventGenerator::default() .with_type_filter(nexmark::event::EventType::Bid) - .map(|e| match e { - nexmark::event::Event::Bid(bid) => bid, // extract the bid event - _ => unreachable!(), - }) .enumerate(); let message_base = SourceMessage { @@ -59,8 +56,15 @@ fn make_batch() -> Vec { generator .by_ref() .take(16384) - .map(|(i, e)| { - let payload = serde_json::to_vec(&e).unwrap(); + .map(|(i, event)| { + let payload = if use_struct { + serde_json::to_vec(&event).unwrap() + } else { + let nexmark::event::Event::Bid(bid) = event else { + unreachable!() + }; + serde_json::to_vec(&bid).unwrap() + }; SourceMessage { payload: Some(payload), offset: i.to_string(), @@ -70,14 +74,18 @@ fn make_batch() -> Vec { .collect_vec() } -fn make_data_stream() -> BoxSourceStream { - futures::future::ready(Ok(BATCH.clone())) - .into_stream() - .boxed() +fn make_data_stream(use_struct: bool) -> BoxSourceStream { + futures::future::ready(Ok(if use_struct { + STRUCT_BATCH.clone() + } else { + BATCH.clone() + })) + .into_stream() + .boxed() } -fn make_parser() -> ByteStreamSourceParserImpl { - let rw_columns = [ +fn make_parser(use_struct: bool) -> ByteStreamSourceParserImpl { + let fields = vec![ ("auction", DataType::Int64), ("bidder", DataType::Int64), ("price", DataType::Int64), @@ -85,11 +93,23 @@ fn make_parser() -> ByteStreamSourceParserImpl { ("url", DataType::Varchar), ("date_time", DataType::Timestamp), ("extra", DataType::Varchar), - ] - .into_iter() - .enumerate() - .map(|(i, (n, t))| SourceColumnDesc::simple(n, t, ColumnId::new(i as _))) - .collect_vec(); + ]; + + let rw_columns = if use_struct { + let fields = fields + .into_iter() + .enumerate() + .map(|(i, (n, t))| ColumnDesc::named(n, ColumnId::new(i as _), t)) + .collect(); + let struct_col = ColumnDesc::new_struct("bid", 114514, "bid", fields); + vec![(&struct_col).into()] + } else { + fields + .into_iter() + .enumerate() + .map(|(i, (n, t))| SourceColumnDesc::simple(n, t, ColumnId::new(i as _))) + .collect_vec() + }; let config = ParserConfig { common: CommonParserConfig { rw_columns }, @@ -99,8 +119,10 @@ fn make_parser() -> ByteStreamSourceParserImpl { ByteStreamSourceParserImpl::create_for_test(config).unwrap() } -fn make_stream_iter() -> impl Iterator { - let mut stream: BoxChunkSourceStream = make_parser().into_stream(make_data_stream()).boxed(); +fn make_stream_iter(use_struct: bool) -> impl Iterator { + let mut stream: BoxChunkSourceStream = make_parser(use_struct) + .into_stream(make_data_stream(use_struct)) + .boxed(); std::iter::from_fn(move || { stream @@ -116,7 +138,7 @@ fn make_stream_iter() -> impl Iterator { fn bench(c: &mut Criterion) { c.bench_function("parse_nexmark", |b| { b.iter_batched( - make_stream_iter, + || make_stream_iter(false), |mut iter| iter.next().unwrap(), BatchSize::SmallInput, ) @@ -135,11 +157,19 @@ fn bench(c: &mut Criterion) { .into(); b.iter_batched( - make_stream_iter, + || make_stream_iter(false), |mut iter| tracing::dispatcher::with_default(&dispatch, || iter.next().unwrap()), BatchSize::SmallInput, ) }); + + c.bench_function("parse_nexmark_struct_type", |b| { + b.iter_batched( + || make_stream_iter(true), + |mut iter| iter.next().unwrap(), + BatchSize::SmallInput, + ) + }); } criterion_group!(benches, bench); From 8f663e4ae78d06b52b9ce0d2e5fc0040448048a1 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Tue, 11 Jun 2024 13:22:48 +0800 Subject: [PATCH 04/37] feat(consistency): tolerate inconsistent stream in OverWindow (#17168) Signed-off-by: Richard Chien --- .../src/executor/over_window/frame_finder.rs | 16 +-- .../src/executor/over_window/general.rs | 58 ++++++--- .../executor/over_window/over_partition.rs | 110 ++++++++++++------ 3 files changed, 128 insertions(+), 56 deletions(-) diff --git a/src/stream/src/executor/over_window/frame_finder.rs b/src/stream/src/executor/over_window/frame_finder.rs index 343d9e4b3df79..12e9f21842887 100644 --- a/src/stream/src/executor/over_window/frame_finder.rs +++ b/src/stream/src/executor/over_window/frame_finder.rs @@ -103,7 +103,7 @@ pub(super) fn merge_rows_frames(rows_frames: &[&RowsFrameBounds]) -> RowsFrameBo /// /// More examples can be found in the comment inside [`find_curr_for_rows_frame`]. pub(super) fn find_first_curr_for_rows_frame<'cache>( - frame_bounds: &'_ RowsFrameBounds, + frame_bounds: &RowsFrameBounds, part_with_delta: DeltaBTreeMap<'cache, CacheKey, OwnedRow>, delta_key: &'cache CacheKey, ) -> &'cache CacheKey { @@ -116,7 +116,7 @@ pub(super) fn find_first_curr_for_rows_frame<'cache>( /// /// This is the symmetric function of [`find_first_curr_for_rows_frame`]. pub(super) fn find_last_curr_for_rows_frame<'cache>( - frame_bounds: &'_ RowsFrameBounds, + frame_bounds: &RowsFrameBounds, part_with_delta: DeltaBTreeMap<'cache, CacheKey, OwnedRow>, delta_key: &'cache CacheKey, ) -> &'cache CacheKey { @@ -127,7 +127,7 @@ pub(super) fn find_last_curr_for_rows_frame<'cache>( /// to some CURRENT ROW, find the cache key corresponding to the start row in /// that frame. pub(super) fn find_frame_start_for_rows_frame<'cache>( - frame_bounds: &'_ RowsFrameBounds, + frame_bounds: &RowsFrameBounds, part_with_delta: DeltaBTreeMap<'cache, CacheKey, OwnedRow>, curr_key: &'cache CacheKey, ) -> &'cache CacheKey { @@ -140,7 +140,7 @@ pub(super) fn find_frame_start_for_rows_frame<'cache>( /// /// This is the symmetric function of [`find_frame_start_for_rows_frame`]. pub(super) fn find_frame_end_for_rows_frame<'cache>( - frame_bounds: &'_ RowsFrameBounds, + frame_bounds: &RowsFrameBounds, part_with_delta: DeltaBTreeMap<'cache, CacheKey, OwnedRow>, curr_key: &'cache CacheKey, ) -> &'cache CacheKey { @@ -189,7 +189,7 @@ pub(super) fn find_left_for_range_frames<'cache>( logical_order_value: impl ToDatumRef, cache_key_pk_len: usize, // this is dirty but we have no better choice ) -> &'cache CacheKey { - find_for_range_frames::( + find_for_range_frames::( range_frames, part_with_delta, logical_order_value, @@ -206,7 +206,7 @@ pub(super) fn find_right_for_range_frames<'cache>( logical_order_value: impl ToDatumRef, cache_key_pk_len: usize, // this is dirty but we have no better choice ) -> &'cache CacheKey { - find_for_range_frames::( + find_for_range_frames::( range_frames, part_with_delta, logical_order_value, @@ -217,7 +217,7 @@ pub(super) fn find_right_for_range_frames<'cache>( // -------------------------- ↑ PUBLIC INTERFACE ↑ -------------------------- fn find_curr_for_rows_frame<'cache, const LEFT: bool>( - frame_bounds: &'_ RowsFrameBounds, + frame_bounds: &RowsFrameBounds, part_with_delta: DeltaBTreeMap<'cache, CacheKey, OwnedRow>, delta_key: &'cache CacheKey, ) -> &'cache CacheKey { @@ -329,7 +329,7 @@ fn find_curr_for_rows_frame<'cache, const LEFT: bool>( } fn find_boundary_for_rows_frame<'cache, const LEFT: bool>( - frame_bounds: &'_ RowsFrameBounds, + frame_bounds: &RowsFrameBounds, part_with_delta: DeltaBTreeMap<'cache, CacheKey, OwnedRow>, curr_key: &'cache CacheKey, ) -> &'cache CacheKey { diff --git a/src/stream/src/executor/over_window/general.rs b/src/stream/src/executor/over_window/general.rs index 815e3b6698a51..23623be6e0f2c 100644 --- a/src/stream/src/executor/over_window/general.rs +++ b/src/stream/src/executor/over_window/general.rs @@ -29,7 +29,6 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_expr::window_function::{ create_window_state, StateKey, WindowFuncCall, WindowStates, }; -use risingwave_storage::row_serde::row_serde_util::serialize_pk_with_vnode; use super::over_partition::{ new_empty_partition_cache, shrink_partition_cache, CacheKey, OverPartition, PartitionCache, @@ -37,6 +36,7 @@ use super::over_partition::{ }; use crate::cache::ManagedLruCache; use crate::common::metrics::MetricsInfo; +use crate::consistency::consistency_panic; use crate::executor::monitor::OverWindowMetrics; use crate::executor::over_window::over_partition::AffectedRange; use crate::executor::prelude::*; @@ -212,7 +212,20 @@ impl OverWindowExecutor { new_row: row, }; } - _ => panic!("inconsistent changes in input chunk"), + _ => { + consistency_panic!( + ?pk, + "inconsistent changes in input chunk, double-inserting" + ); + if let Record::Update { old_row, .. } = prev_change { + *prev_change = Record::Update { + old_row: *old_row, + new_row: row, + }; + } else { + *prev_change = Record::Insert { new_row: row }; + } + } } } else { changes_merged.insert(pk, Record::Insert { new_row: row }); @@ -232,7 +245,13 @@ impl OverWindowExecutor { old_row: *real_old_row, }; } - _ => panic!("inconsistent changes in input chunk"), + _ => { + consistency_panic!( + ?pk, + "inconsistent changes in input chunk, double-deleting" + ); + *prev_change = Record::Delete { old_row: row }; + } } } else { changes_merged.insert(pk, Record::Delete { old_row: row }); @@ -357,13 +376,17 @@ impl OverWindowExecutor { } } (existed, record) => { - let vnode = this.state_table.compute_vnode_by_pk(&key.pk); - let raw_key = serialize_pk_with_vnode( - &key.pk, - this.state_table.pk_serde(), - vnode, + // when stream is inconsistent, there may be an `Update` of which the old pk does not actually exist + consistency_panic!( + ?existed, + ?record, + "other cases should not exist", ); - panic!("other cases should not exist. raw_key: {:?}, existed: {:?}, new: {:?}", raw_key, existed, record); + + key_change_update_buffer.insert(pk, record); + if let Some(chunk) = chunk_builder.append_record(existed) { + yield chunk; + } } } } else { @@ -375,6 +398,15 @@ impl OverWindowExecutor { partition.write_record(&mut this.state_table, key, record); } + if !key_change_update_buffer.is_empty() { + consistency_panic!( + ?key_change_update_buffer, + "key-change update buffer should be empty after processing" + ); + // if in non-strict mode, we can reach here, but we don't know the `StateKey`, + // so just ignore the buffer. + } + let cache_len = partition.cache_real_len(); let stats = partition.summarize(); metrics @@ -423,21 +455,17 @@ impl OverWindowExecutor { async fn build_changes_for_partition( this: &ExecutorInner, partition: &mut OverPartition<'_, S>, - delta: PartitionDelta, + mut delta: PartitionDelta, ) -> StreamExecutorResult<( BTreeMap>, Option>, )> { - assert!(!delta.is_empty(), "if there's no delta, we won't be here"); - let mut part_changes = BTreeMap::new(); // Find affected ranges, this also ensures that all rows in the affected ranges are loaded // into the cache. - // TODO(rc): maybe we can find affected ranges for each window function call (each frame) to simplify - // the implementation of `find_affected_ranges` let (part_with_delta, affected_ranges) = partition - .find_affected_ranges(&this.state_table, &delta) + .find_affected_ranges(&this.state_table, &mut delta) .await?; let snapshot = part_with_delta.snapshot(); diff --git a/src/stream/src/executor/over_window/over_partition.rs b/src/stream/src/executor/over_window/over_partition.rs index a3672fc2061e6..a00f86aff9082 100644 --- a/src/stream/src/executor/over_window/over_partition.rs +++ b/src/stream/src/executor/over_window/over_partition.rs @@ -36,6 +36,7 @@ use static_assertions::const_assert; use super::general::RowConverter; use crate::common::table::state_table::StateTable; +use crate::consistency::{consistency_error, enable_strict_consistency}; use crate::executor::over_window::frame_finder::*; use crate::executor::StreamExecutorResult; @@ -255,19 +256,19 @@ pub(super) struct OverPartitionStats { /// included for computing the new state. #[derive(Debug, Educe)] #[educe(Clone, Copy)] -pub(super) struct AffectedRange<'cache> { - pub first_frame_start: &'cache CacheKey, - pub first_curr_key: &'cache CacheKey, - pub last_curr_key: &'cache CacheKey, - pub last_frame_end: &'cache CacheKey, +pub(super) struct AffectedRange<'a> { + pub first_frame_start: &'a CacheKey, + pub first_curr_key: &'a CacheKey, + pub last_curr_key: &'a CacheKey, + pub last_frame_end: &'a CacheKey, } -impl<'cache> AffectedRange<'cache> { +impl<'a> AffectedRange<'a> { fn new( - first_frame_start: &'cache CacheKey, - first_curr_key: &'cache CacheKey, - last_curr_key: &'cache CacheKey, - last_frame_end: &'cache CacheKey, + first_frame_start: &'a CacheKey, + first_curr_key: &'a CacheKey, + last_curr_key: &'a CacheKey, + last_frame_end: &'a CacheKey, ) -> Self { Self { first_frame_start, @@ -436,35 +437,31 @@ impl<'a, S: StateStore> OverPartition<'a, S> { /// Find all ranges in the partition that are affected by the given delta. /// The returned ranges are guaranteed to be sorted and non-overlapping. All keys in the ranges /// are guaranteed to be cached, which means they should be [`Sentinelled::Normal`]s. - pub async fn find_affected_ranges<'s, 'cache>( + pub async fn find_affected_ranges<'s, 'delta>( &'s mut self, - table: &'_ StateTable, - delta: &'cache PartitionDelta, + table: &StateTable, + delta: &'delta mut PartitionDelta, ) -> StreamExecutorResult<( - DeltaBTreeMap<'cache, CacheKey, OwnedRow>, - Vec>, + DeltaBTreeMap<'delta, CacheKey, OwnedRow>, + Vec>, )> where - 's: 'cache, + 'a: 'delta, + 's: 'delta, { + self.ensure_delta_in_cache(table, delta).await?; + let delta = &*delta; // let's make it immutable + + if delta.is_empty() { + return Ok((DeltaBTreeMap::new(self.range_cache.inner(), delta), vec![])); + } + let delta_first = delta.first_key_value().unwrap().0.as_normal_expect(); let delta_last = delta.last_key_value().unwrap().0.as_normal_expect(); let range_frame_logical_curr = calc_logical_curr_for_range_frames(&self.range_frames, delta_first, delta_last); - if self.cache_policy.is_full() { - // ensure everything is in the cache - self.extend_cache_to_boundary(table).await?; - } else { - // TODO(rc): later we should extend cache using `self.super_rows_frame_bounds` and - // `range_frame_logical_curr` as hints. - - // ensure the cache covers all delta (if possible) - self.extend_cache_by_range(table, delta_first..=delta_last) - .await?; - } - loop { // TERMINATEABILITY: `extend_cache_leftward_by_n` and `extend_cache_rightward_by_n` keep // pushing the cache to the boundary of current partition. In these two methods, when @@ -472,7 +469,7 @@ impl<'a, S: StateStore> OverPartition<'a, S> { // `Self::find_affected_ranges_readonly` will return `Ok`. // SAFETY: Here we shortly borrow the range cache and turn the reference into a - // `'cache` one to bypass the borrow checker. This is safe because we only return + // `'delta` one to bypass the borrow checker. This is safe because we only return // the reference once we don't need to do any further mutation. let cache_inner = unsafe { &*(self.range_cache.inner() as *const _) }; let part_with_delta = DeltaBTreeMap::new(cache_inner, delta); @@ -502,6 +499,53 @@ impl<'a, S: StateStore> OverPartition<'a, S> { } } + async fn ensure_delta_in_cache( + &mut self, + table: &StateTable, + delta: &mut PartitionDelta, + ) -> StreamExecutorResult<()> { + if delta.is_empty() { + return Ok(()); + } + + let delta_first = delta.first_key_value().unwrap().0.as_normal_expect(); + let delta_last = delta.last_key_value().unwrap().0.as_normal_expect(); + + if self.cache_policy.is_full() { + // ensure everything is in the cache + self.extend_cache_to_boundary(table).await?; + } else { + // TODO(rc): later we should extend cache using `self.super_rows_frame_bounds` and + // `range_frame_logical_curr` as hints. + + // ensure the cache covers all delta (if possible) + self.extend_cache_by_range(table, delta_first..=delta_last) + .await?; + } + + if !enable_strict_consistency() { + // in non-strict mode, we should ensure the delta is consistent with the cache + let cache = self.range_cache.inner(); + delta.retain(|key, change| match &*change { + Change::Insert(_) => { + // this also includes the case of double-insert and ghost-update, + // but since we already lost the information, let's just ignore it + true + } + Change::Delete => { + // if the key is not in the cache, it's a ghost-delete + let consistent = cache.contains_key(key); + if !consistent { + consistency_error!(?key, "removing a row with non-existing key"); + } + consistent + } + }); + } + + Ok(()) + } + /// Try to find affected ranges on immutable range cache + delta. If the algorithm reaches /// any sentinel node in the cache, which means some entries in the affected range may be /// in the state table, it returns an `Err((bool, bool))` to notify the caller that the @@ -510,11 +554,11 @@ impl<'a, S: StateStore> OverPartition<'a, S> { /// TODO(rc): Currently at most one range will be in the result vector. Ideally we should /// recognize uncontinuous changes in the delta and find multiple ranges, but that will be /// too complex for now. - fn find_affected_ranges_readonly<'cache>( - &'_ self, - part_with_delta: DeltaBTreeMap<'cache, CacheKey, OwnedRow>, + fn find_affected_ranges_readonly<'delta>( + &self, + part_with_delta: DeltaBTreeMap<'delta, CacheKey, OwnedRow>, range_frame_logical_curr: Option<&(Sentinelled, Sentinelled)>, - ) -> std::result::Result>, (bool, bool)> { + ) -> std::result::Result>, (bool, bool)> { if part_with_delta.first_key().is_none() { // nothing is left after applying the delta, meaning all entries are deleted return Ok(vec![]); From 549cbf565070052c70108aac4d06c96fffb48472 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 11 Jun 2024 13:32:12 +0800 Subject: [PATCH 05/37] refactor(test): move backwards compat test and slow tests into e2e_test folder (#17190) --- Makefile.toml | 2 +- ci/scripts/backwards-compat-test.sh | 2 +- ci/scripts/slow-e2e-test.sh | 4 ++-- .../backwards-compat-tests}/README.md | 0 .../backwards-compat-tests}/scripts/run_local.sh | 2 +- .../backwards-compat-tests}/scripts/utils.sh | 4 ++-- .../backwards-compat-tests}/slt/basic/seed.slt | 0 .../backwards-compat-tests}/slt/basic/validate_original.slt | 0 .../backwards-compat-tests}/slt/basic/validate_restart.slt | 0 .../slt/kafka/invalid_options/seed.slt | 0 .../slt/kafka/invalid_options/validate_original.slt | 0 .../slt/kafka/invalid_options/validate_restart.slt | 0 .../backwards-compat-tests}/slt/kafka/seed.slt | 0 .../slt/kafka/upsert/deprecate_upsert.slt | 0 .../slt/kafka/upsert/include_key_as.slt | 0 .../backwards-compat-tests}/slt/kafka/validate_original.slt | 0 .../backwards-compat-tests}/slt/kafka/validate_restart.slt | 0 .../slt/nexmark-backwards-compat/delete.slt.part | 0 .../slt/nexmark-backwards-compat/insert.slt | 0 .../slt/nexmark-backwards-compat/seed.slt | 0 .../slt/nexmark-backwards-compat/validate_original.slt | 0 .../slt/nexmark-backwards-compat/validate_restart.slt | 0 .../slt/tpch-backwards-compat/delete.slt.part | 0 .../slt/tpch-backwards-compat/insert.slt.part | 0 .../slt/tpch-backwards-compat/seed.slt | 0 .../slt/tpch-backwards-compat/validate_original.slt | 0 .../slt/tpch-backwards-compat/validate_restart.slt | 0 .../slow_tests}/backfill/rate_limit/slow-udf.slt | 0 .../slow_tests}/udf/always_retry_python.slt | 0 29 files changed, 7 insertions(+), 7 deletions(-) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/README.md (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/scripts/run_local.sh (96%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/scripts/utils.sh (98%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/basic/seed.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/basic/validate_original.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/basic/validate_restart.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/kafka/invalid_options/seed.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/kafka/invalid_options/validate_original.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/kafka/invalid_options/validate_restart.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/kafka/seed.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/kafka/upsert/deprecate_upsert.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/kafka/upsert/include_key_as.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/kafka/validate_original.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/kafka/validate_restart.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/nexmark-backwards-compat/delete.slt.part (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/nexmark-backwards-compat/insert.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/nexmark-backwards-compat/seed.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/nexmark-backwards-compat/validate_original.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/nexmark-backwards-compat/validate_restart.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/tpch-backwards-compat/delete.slt.part (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/tpch-backwards-compat/insert.slt.part (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/tpch-backwards-compat/seed.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/tpch-backwards-compat/validate_original.slt (100%) rename {backwards-compat-tests => e2e_test/backwards-compat-tests}/slt/tpch-backwards-compat/validate_restart.slt (100%) rename {e2e_test_slow => e2e_test/slow_tests}/backfill/rate_limit/slow-udf.slt (100%) rename {e2e_test_slow => e2e_test/slow_tests}/udf/always_retry_python.slt (100%) diff --git a/Makefile.toml b/Makefile.toml index a154f0ba60dba..1e618b1aed6ba 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -1417,7 +1417,7 @@ UPDATE_EXPECT=1 cargo test -p risingwave_connector tests::test_with_options_yaml [tasks.backwards-compat-test] category = "RiseDev - Test - Backwards Compatibility Test" description = "Run backwards compatibility test" -script = "./backwards-compat-tests/scripts/run_local.sh" +script = "./e2e_test/backwards-compat-tests/scripts/run_local.sh" # For debugging. # To show the env for a specific task, use `run_task = "show-env"` for that task. diff --git a/ci/scripts/backwards-compat-test.sh b/ci/scripts/backwards-compat-test.sh index 9089be7a1bdc1..90a01f57a51cc 100755 --- a/ci/scripts/backwards-compat-test.sh +++ b/ci/scripts/backwards-compat-test.sh @@ -32,7 +32,7 @@ else exit 1 fi -source backwards-compat-tests/scripts/utils.sh +source e2e_test/backwards-compat-tests/scripts/utils.sh ################################### Main diff --git a/ci/scripts/slow-e2e-test.sh b/ci/scripts/slow-e2e-test.sh index 2fa1c977a9cfa..dc33c2719053b 100755 --- a/ci/scripts/slow-e2e-test.sh +++ b/ci/scripts/slow-e2e-test.sh @@ -36,5 +36,5 @@ echo "--- e2e, $mode, slow-udf-tests" python3 -m pip install --break-system-packages arrow-udf==0.2.1 RUST_LOG="info" \ risedev ci-start "$mode" -sqllogictest -p 4566 -d dev './e2e_test_slow/udf/always_retry_python.slt' -sqllogictest -p 4566 -d dev './e2e_test_slow/backfill/rate_limit/slow-udf.slt' +sqllogictest -p 4566 -d dev './e2e_test/slow_tests/udf/always_retry_python.slt' +sqllogictest -p 4566 -d dev './e2e_test/slow_tests/backfill/rate_limit/slow-udf.slt' diff --git a/backwards-compat-tests/README.md b/e2e_test/backwards-compat-tests/README.md similarity index 100% rename from backwards-compat-tests/README.md rename to e2e_test/backwards-compat-tests/README.md diff --git a/backwards-compat-tests/scripts/run_local.sh b/e2e_test/backwards-compat-tests/scripts/run_local.sh similarity index 96% rename from backwards-compat-tests/scripts/run_local.sh rename to e2e_test/backwards-compat-tests/scripts/run_local.sh index e2604e6a114e0..7ac952e6d1c4a 100755 --- a/backwards-compat-tests/scripts/run_local.sh +++ b/e2e_test/backwards-compat-tests/scripts/run_local.sh @@ -10,7 +10,7 @@ on_exit() { trap on_exit EXIT -source backwards-compat-tests/scripts/utils.sh +source e2e_test/backwards-compat-tests/scripts/utils.sh configure_rw() { VERSION="$1" diff --git a/backwards-compat-tests/scripts/utils.sh b/e2e_test/backwards-compat-tests/scripts/utils.sh similarity index 98% rename from backwards-compat-tests/scripts/utils.sh rename to e2e_test/backwards-compat-tests/scripts/utils.sh index 6b5184b2bb0ea..8f41dad0860f1 100644 --- a/backwards-compat-tests/scripts/utils.sh +++ b/e2e_test/backwards-compat-tests/scripts/utils.sh @@ -18,9 +18,9 @@ RECOVERY_DURATION=20 # Setup test directory -TEST_DIR=.risingwave/backwards-compat-tests/ +TEST_DIR=.risingwave/e2e_test/backwards-compat-tests/ mkdir -p $TEST_DIR -cp -r backwards-compat-tests/slt/* $TEST_DIR +cp -r e2e_test/backwards-compat-tests/slt/* $TEST_DIR wait_for_process() { process_name="$1" diff --git a/backwards-compat-tests/slt/basic/seed.slt b/e2e_test/backwards-compat-tests/slt/basic/seed.slt similarity index 100% rename from backwards-compat-tests/slt/basic/seed.slt rename to e2e_test/backwards-compat-tests/slt/basic/seed.slt diff --git a/backwards-compat-tests/slt/basic/validate_original.slt b/e2e_test/backwards-compat-tests/slt/basic/validate_original.slt similarity index 100% rename from backwards-compat-tests/slt/basic/validate_original.slt rename to e2e_test/backwards-compat-tests/slt/basic/validate_original.slt diff --git a/backwards-compat-tests/slt/basic/validate_restart.slt b/e2e_test/backwards-compat-tests/slt/basic/validate_restart.slt similarity index 100% rename from backwards-compat-tests/slt/basic/validate_restart.slt rename to e2e_test/backwards-compat-tests/slt/basic/validate_restart.slt diff --git a/backwards-compat-tests/slt/kafka/invalid_options/seed.slt b/e2e_test/backwards-compat-tests/slt/kafka/invalid_options/seed.slt similarity index 100% rename from backwards-compat-tests/slt/kafka/invalid_options/seed.slt rename to e2e_test/backwards-compat-tests/slt/kafka/invalid_options/seed.slt diff --git a/backwards-compat-tests/slt/kafka/invalid_options/validate_original.slt b/e2e_test/backwards-compat-tests/slt/kafka/invalid_options/validate_original.slt similarity index 100% rename from backwards-compat-tests/slt/kafka/invalid_options/validate_original.slt rename to e2e_test/backwards-compat-tests/slt/kafka/invalid_options/validate_original.slt diff --git a/backwards-compat-tests/slt/kafka/invalid_options/validate_restart.slt b/e2e_test/backwards-compat-tests/slt/kafka/invalid_options/validate_restart.slt similarity index 100% rename from backwards-compat-tests/slt/kafka/invalid_options/validate_restart.slt rename to e2e_test/backwards-compat-tests/slt/kafka/invalid_options/validate_restart.slt diff --git a/backwards-compat-tests/slt/kafka/seed.slt b/e2e_test/backwards-compat-tests/slt/kafka/seed.slt similarity index 100% rename from backwards-compat-tests/slt/kafka/seed.slt rename to e2e_test/backwards-compat-tests/slt/kafka/seed.slt diff --git a/backwards-compat-tests/slt/kafka/upsert/deprecate_upsert.slt b/e2e_test/backwards-compat-tests/slt/kafka/upsert/deprecate_upsert.slt similarity index 100% rename from backwards-compat-tests/slt/kafka/upsert/deprecate_upsert.slt rename to e2e_test/backwards-compat-tests/slt/kafka/upsert/deprecate_upsert.slt diff --git a/backwards-compat-tests/slt/kafka/upsert/include_key_as.slt b/e2e_test/backwards-compat-tests/slt/kafka/upsert/include_key_as.slt similarity index 100% rename from backwards-compat-tests/slt/kafka/upsert/include_key_as.slt rename to e2e_test/backwards-compat-tests/slt/kafka/upsert/include_key_as.slt diff --git a/backwards-compat-tests/slt/kafka/validate_original.slt b/e2e_test/backwards-compat-tests/slt/kafka/validate_original.slt similarity index 100% rename from backwards-compat-tests/slt/kafka/validate_original.slt rename to e2e_test/backwards-compat-tests/slt/kafka/validate_original.slt diff --git a/backwards-compat-tests/slt/kafka/validate_restart.slt b/e2e_test/backwards-compat-tests/slt/kafka/validate_restart.slt similarity index 100% rename from backwards-compat-tests/slt/kafka/validate_restart.slt rename to e2e_test/backwards-compat-tests/slt/kafka/validate_restart.slt diff --git a/backwards-compat-tests/slt/nexmark-backwards-compat/delete.slt.part b/e2e_test/backwards-compat-tests/slt/nexmark-backwards-compat/delete.slt.part similarity index 100% rename from backwards-compat-tests/slt/nexmark-backwards-compat/delete.slt.part rename to e2e_test/backwards-compat-tests/slt/nexmark-backwards-compat/delete.slt.part diff --git a/backwards-compat-tests/slt/nexmark-backwards-compat/insert.slt b/e2e_test/backwards-compat-tests/slt/nexmark-backwards-compat/insert.slt similarity index 100% rename from backwards-compat-tests/slt/nexmark-backwards-compat/insert.slt rename to e2e_test/backwards-compat-tests/slt/nexmark-backwards-compat/insert.slt diff --git a/backwards-compat-tests/slt/nexmark-backwards-compat/seed.slt b/e2e_test/backwards-compat-tests/slt/nexmark-backwards-compat/seed.slt similarity index 100% rename from backwards-compat-tests/slt/nexmark-backwards-compat/seed.slt rename to e2e_test/backwards-compat-tests/slt/nexmark-backwards-compat/seed.slt diff --git a/backwards-compat-tests/slt/nexmark-backwards-compat/validate_original.slt b/e2e_test/backwards-compat-tests/slt/nexmark-backwards-compat/validate_original.slt similarity index 100% rename from backwards-compat-tests/slt/nexmark-backwards-compat/validate_original.slt rename to e2e_test/backwards-compat-tests/slt/nexmark-backwards-compat/validate_original.slt diff --git a/backwards-compat-tests/slt/nexmark-backwards-compat/validate_restart.slt b/e2e_test/backwards-compat-tests/slt/nexmark-backwards-compat/validate_restart.slt similarity index 100% rename from backwards-compat-tests/slt/nexmark-backwards-compat/validate_restart.slt rename to e2e_test/backwards-compat-tests/slt/nexmark-backwards-compat/validate_restart.slt diff --git a/backwards-compat-tests/slt/tpch-backwards-compat/delete.slt.part b/e2e_test/backwards-compat-tests/slt/tpch-backwards-compat/delete.slt.part similarity index 100% rename from backwards-compat-tests/slt/tpch-backwards-compat/delete.slt.part rename to e2e_test/backwards-compat-tests/slt/tpch-backwards-compat/delete.slt.part diff --git a/backwards-compat-tests/slt/tpch-backwards-compat/insert.slt.part b/e2e_test/backwards-compat-tests/slt/tpch-backwards-compat/insert.slt.part similarity index 100% rename from backwards-compat-tests/slt/tpch-backwards-compat/insert.slt.part rename to e2e_test/backwards-compat-tests/slt/tpch-backwards-compat/insert.slt.part diff --git a/backwards-compat-tests/slt/tpch-backwards-compat/seed.slt b/e2e_test/backwards-compat-tests/slt/tpch-backwards-compat/seed.slt similarity index 100% rename from backwards-compat-tests/slt/tpch-backwards-compat/seed.slt rename to e2e_test/backwards-compat-tests/slt/tpch-backwards-compat/seed.slt diff --git a/backwards-compat-tests/slt/tpch-backwards-compat/validate_original.slt b/e2e_test/backwards-compat-tests/slt/tpch-backwards-compat/validate_original.slt similarity index 100% rename from backwards-compat-tests/slt/tpch-backwards-compat/validate_original.slt rename to e2e_test/backwards-compat-tests/slt/tpch-backwards-compat/validate_original.slt diff --git a/backwards-compat-tests/slt/tpch-backwards-compat/validate_restart.slt b/e2e_test/backwards-compat-tests/slt/tpch-backwards-compat/validate_restart.slt similarity index 100% rename from backwards-compat-tests/slt/tpch-backwards-compat/validate_restart.slt rename to e2e_test/backwards-compat-tests/slt/tpch-backwards-compat/validate_restart.slt diff --git a/e2e_test_slow/backfill/rate_limit/slow-udf.slt b/e2e_test/slow_tests/backfill/rate_limit/slow-udf.slt similarity index 100% rename from e2e_test_slow/backfill/rate_limit/slow-udf.slt rename to e2e_test/slow_tests/backfill/rate_limit/slow-udf.slt diff --git a/e2e_test_slow/udf/always_retry_python.slt b/e2e_test/slow_tests/udf/always_retry_python.slt similarity index 100% rename from e2e_test_slow/udf/always_retry_python.slt rename to e2e_test/slow_tests/udf/always_retry_python.slt From 6cd0f2806270f10f8365153fa1e03e6f3d41828c Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 11 Jun 2024 14:19:45 +0800 Subject: [PATCH 06/37] refactor(connector): adopt zero-copy access for protobuf & avro parsing (#17171) Signed-off-by: Bugen Zhao --- src/common/src/types/cow.rs | 5 ++ src/common/src/types/jsonb.rs | 7 ++- src/connector/codec/src/decoder/avro/mod.rs | 45 ++++++++++------- src/connector/src/parser/protobuf/parser.rs | 53 +++++++++++++------- src/connector/src/parser/unified/json.rs | 6 +-- src/connector/src/parser/unified/protobuf.rs | 14 ++++-- 6 files changed, 84 insertions(+), 46 deletions(-) diff --git a/src/common/src/types/cow.rs b/src/common/src/types/cow.rs index 928af654b0366..88eebfe1e4c2a 100644 --- a/src/common/src/types/cow.rs +++ b/src/common/src/types/cow.rs @@ -73,3 +73,8 @@ impl ToOwnedDatum for DatumCow<'_> { } } } + +impl DatumCow<'_> { + /// Equivalent to `DatumCow::Owned(Datum::None)`. + pub const NULL: DatumCow<'static> = DatumCow::Owned(None); +} diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 824020fac3123..642b363a8c67e 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -239,10 +239,15 @@ impl<'a> JsonbRef<'a> { } /// Returns a jsonb `null` value. - pub fn null() -> Self { + pub const fn null() -> Self { Self(ValueRef::Null) } + /// Returns a value for empty string. + pub const fn empty_string() -> Self { + Self(ValueRef::String("")) + } + /// Returns true if this is a jsonb `null`. pub fn is_jsonb_null(&self) -> bool { self.0.is_null() diff --git a/src/connector/codec/src/decoder/avro/mod.rs b/src/connector/codec/src/decoder/avro/mod.rs index e48eecfb7e2cf..cdd9aea416c8f 100644 --- a/src/connector/codec/src/decoder/avro/mod.rs +++ b/src/connector/codec/src/decoder/avro/mod.rs @@ -26,6 +26,7 @@ use risingwave_common::bail; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{ DataType, Date, DatumCow, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, + ToOwnedDatum, }; use risingwave_common::util::iter_util::ZipEqFast; @@ -56,7 +57,7 @@ impl<'a> AvroParseOptions<'a> { } impl<'a> AvroParseOptions<'a> { - fn extract_inner_schema(&self, key: Option<&'a str>) -> Option<&'a Schema> { + fn extract_inner_schema(&self, key: Option<&str>) -> Option<&'a Schema> { self.schema .map(|schema| avro_extract_field_schema(schema, key)) .transpose() @@ -87,8 +88,8 @@ impl<'a> AvroParseOptions<'a> { pub fn convert_to_datum<'b>( &self, value: &'b Value, - type_expected: &'b DataType, - ) -> AccessResult + type_expected: &DataType, + ) -> AccessResult> where 'b: 'a, { @@ -98,8 +99,14 @@ impl<'a> AvroParseOptions<'a> { value: String::new(), }; + macro_rules! borrowed { + ($v:expr) => { + return Ok(DatumCow::Borrowed(Some($v.into()))) + }; + } + let v: ScalarImpl = match (type_expected, value) { - (_, Value::Null) => return Ok(None), + (_, Value::Null) => return Ok(DatumCow::NULL), (_, Value::Union(_, v)) => { let schema = self.extract_inner_schema(None); return Self { @@ -183,8 +190,8 @@ impl<'a> AvroParseOptions<'a> { .map_err(|_| create_error())? .into(), // ---- Varchar ----- - (DataType::Varchar, Value::Enum(_, symbol)) => symbol.clone().into_boxed_str().into(), - (DataType::Varchar, Value::String(s)) => s.clone().into_boxed_str().into(), + (DataType::Varchar, Value::Enum(_, symbol)) => borrowed!(symbol.as_str()), + (DataType::Varchar, Value::String(s)) => borrowed!(s.as_str()), // ---- Timestamp ----- (DataType::Timestamp, Value::LocalTimestampMillis(ms)) => Timestamp::with_millis(*ms) .map_err(|_| create_error())? @@ -223,7 +230,8 @@ impl<'a> AvroParseOptions<'a> { schema, relax_numeric: self.relax_numeric, } - .convert_to_datum(value, field_type)?) + .convert_to_datum(value, field_type)? + .to_owned_datum()) } else { Ok(None) } @@ -247,7 +255,7 @@ impl<'a> AvroParseOptions<'a> { }) .into(), // ---- Bytea ----- - (DataType::Bytea, Value::Bytes(value)) => value.clone().into_boxed_slice().into(), + (DataType::Bytea, Value::Bytes(value)) => borrowed!(value.as_slice()), // ---- Jsonb ----- (DataType::Jsonb, v @ Value::Map(_)) => { let mut builder = jsonbb::Builder::default(); @@ -262,7 +270,7 @@ impl<'a> AvroParseOptions<'a> { (_expected, _got) => Err(create_error())?, }; - Ok(Some(v)) + Ok(DatumCow::Owned(Some(v))) } } @@ -308,10 +316,7 @@ impl Access for AvroAccess<'_> { Err(create_error())?; } - // TODO: may borrow the value directly - options - .convert_to_datum(value, type_expected) - .map(Into::into) + options.convert_to_datum(value, type_expected) } } @@ -352,7 +357,7 @@ pub fn avro_schema_skip_union(schema: &Schema) -> anyhow::Result<&Schema> { // extract inner filed/item schema of record/array/union pub fn avro_extract_field_schema<'a>( schema: &'a Schema, - name: Option<&'a str>, + name: Option<&str>, ) -> anyhow::Result<&'a Schema> { match schema { Schema::Record(RecordSchema { fields, lookup, .. }) => { @@ -524,9 +529,9 @@ mod tests { value_schema: &Schema, shape: &DataType, ) -> anyhow::Result { - AvroParseOptions::create(value_schema) - .convert_to_datum(&value, shape) - .map_err(Into::into) + Ok(AvroParseOptions::create(value_schema) + .convert_to_datum(&value, shape)? + .to_owned_datum()) } #[test] @@ -569,7 +574,8 @@ mod tests { let options = AvroParseOptions::create(&schema); let resp = options .convert_to_datum(&value, &DataType::Decimal) - .unwrap(); + .unwrap() + .to_owned_datum(); assert_eq!( resp, Some(ScalarImpl::Decimal(Decimal::Normalized( @@ -608,7 +614,8 @@ mod tests { let options = AvroParseOptions::create(&schema); let resp = options .convert_to_datum(&value, &DataType::Decimal) - .unwrap(); + .unwrap() + .to_owned_datum(); assert_eq!(resp, Some(ScalarImpl::Decimal(Decimal::from(66051)))); } } diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index b5df0aeb83909..8be25074f6295 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -21,7 +21,10 @@ use prost_reflect::{ MessageDescriptor, ReflectMessage, Value, }; use risingwave_common::array::{ListValue, StructValue}; -use risingwave_common::types::{DataType, Datum, Decimal, JsonbVal, ScalarImpl, F32, F64}; +use risingwave_common::types::{ + DataType, Datum, DatumCow, Decimal, JsonbRef, JsonbVal, ScalarImpl, ScalarRefImpl, ToDatumRef, + ToOwnedDatum, F32, F64, +}; use risingwave_common::{bail, try_match_expand}; use risingwave_pb::plan_common::{AdditionalColumn, ColumnDesc, ColumnDescVersion}; use thiserror::Error; @@ -344,14 +347,20 @@ fn recursive_parse_json( serde_json::Value::Object(ret) } -pub fn from_protobuf_value( +pub fn from_protobuf_value<'a>( field_desc: &FieldDescriptor, - value: &Value, + value: &'a Value, descriptor_pool: &Arc, -) -> AccessResult { +) -> AccessResult> { let kind = field_desc.kind(); - let v = match value { + macro_rules! borrowed { + ($v:expr) => { + return Ok(DatumCow::Borrowed(Some($v.into()))) + }; + } + + let v: ScalarImpl = match value { Value::Bool(v) => ScalarImpl::Bool(*v), Value::I32(i) => ScalarImpl::Int32(*i), Value::U32(i) => ScalarImpl::Int64(*i as i64), @@ -359,7 +368,7 @@ pub fn from_protobuf_value( Value::U64(i) => ScalarImpl::Decimal(Decimal::from(*i)), Value::F32(f) => ScalarImpl::Float32(F32::from(*f)), Value::F64(f) => ScalarImpl::Float64(F64::from(*f)), - Value::String(s) => ScalarImpl::Utf8(s.as_str().into()), + Value::String(s) => borrowed!(s.as_str()), Value::EnumNumber(idx) => { let enum_desc = kind.as_enum().ok_or_else(|| AccessError::TypeError { expected: "enum".to_owned(), @@ -375,9 +384,7 @@ pub fn from_protobuf_value( if dyn_msg.descriptor().full_name() == "google.protobuf.Any" { // If the fields are not presented, default value is an empty string if !dyn_msg.has_field_by_name("type_url") || !dyn_msg.has_field_by_name("value") { - return Ok(Some(ScalarImpl::Jsonb(JsonbVal::from( - serde_json::json! {""}, - )))); + borrowed!(JsonbRef::empty_string()); } // Sanity check @@ -391,9 +398,8 @@ pub fn from_protobuf_value( let payload_field_desc = dyn_msg.descriptor().get_field_by_name("value").unwrap(); - let Some(ScalarImpl::Bytea(payload)) = - from_protobuf_value(&payload_field_desc, &payload, descriptor_pool)? - else { + let payload = from_protobuf_value(&payload_field_desc, &payload, descriptor_pool)?; + let Some(ScalarRefImpl::Bytea(payload)) = payload.to_datum_ref() else { bail_uncategorized!("expected bytes for dynamic message payload"); }; @@ -413,12 +419,13 @@ pub fn from_protobuf_value( let full_name = msg_desc.clone().full_name().to_string(); // Decode the payload based on the `msg_desc` - let decoded_value = DynamicMessage::decode(msg_desc, payload.as_ref()).unwrap(); + let decoded_value = DynamicMessage::decode(msg_desc, payload).unwrap(); let decoded_value = from_protobuf_value( field_desc, &Value::Message(decoded_value), descriptor_pool, )? + .to_owned_datum() .unwrap(); // Extract the struct value @@ -447,7 +454,9 @@ pub fn from_protobuf_value( } // use default value if dyn_msg doesn't has this field let value = dyn_msg.get_field(&field_desc); - rw_values.push(from_protobuf_value(&field_desc, &value, descriptor_pool)?); + rw_values.push( + from_protobuf_value(&field_desc, &value, descriptor_pool)?.to_owned_datum(), + ); } ScalarImpl::Struct(StructValue::new(rw_values)) } @@ -461,14 +470,14 @@ pub fn from_protobuf_value( } ScalarImpl::List(ListValue::new(builder.finish())) } - Value::Bytes(value) => ScalarImpl::Bytea(value.to_vec().into_boxed_slice()), + Value::Bytes(value) => borrowed!(&**value), _ => { return Err(AccessError::UnsupportedType { ty: format!("{kind:?}"), }); } }; - Ok(Some(v)) + Ok(Some(v).into()) } /// Maps protobuf type to RW type. @@ -965,7 +974,9 @@ mod test { let field = value.fields().next().unwrap().0; if let Some(ret) = - from_protobuf_value(&field, &Value::Message(value), &conf.descriptor_pool).unwrap() + from_protobuf_value(&field, &Value::Message(value), &conf.descriptor_pool) + .unwrap() + .to_owned_datum() { println!("Decoded Value for ANY_GEN_PROTO_DATA: {:#?}", ret); println!("---------------------------"); @@ -1026,7 +1037,9 @@ mod test { let field = value.fields().next().unwrap().0; if let Some(ret) = - from_protobuf_value(&field, &Value::Message(value), &conf.descriptor_pool).unwrap() + from_protobuf_value(&field, &Value::Message(value), &conf.descriptor_pool) + .unwrap() + .to_owned_datum() { println!("Decoded Value for ANY_GEN_PROTO_DATA: {:#?}", ret); println!("---------------------------"); @@ -1098,7 +1111,9 @@ mod test { let field = value.fields().next().unwrap().0; if let Some(ret) = - from_protobuf_value(&field, &Value::Message(value), &conf.descriptor_pool).unwrap() + from_protobuf_value(&field, &Value::Message(value), &conf.descriptor_pool) + .unwrap() + .to_owned_datum() { println!("Decoded Value for ANY_RECURSIVE_GEN_PROTO_DATA: {:#?}", ret); println!("---------------------------"); diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index fdf61c581b731..09704d9192a41 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -22,8 +22,8 @@ use risingwave_common::array::{ListValue, StructValue}; use risingwave_common::cast::{i64_to_timestamp, i64_to_timestamptz, str_to_bytea}; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{ - DataType, Date, Datum, Decimal, Int256, Interval, JsonbVal, ScalarImpl, Time, Timestamp, - Timestamptz, ToOwnedDatum, + DataType, Date, Decimal, Int256, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, + ToOwnedDatum, }; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_connector_codec::decoder::utils::extract_decimal; @@ -213,7 +213,7 @@ impl JsonParseOptions { }; let v: ScalarImpl = match (type_expected, value.value_type()) { - (_, ValueType::Null) => return Ok(Datum::None.into()), + (_, ValueType::Null) => return Ok(DatumCow::NULL), // ---- Boolean ----- (DataType::Boolean, ValueType::Bool) => value.as_bool().unwrap().into(), diff --git a/src/connector/src/parser/unified/protobuf.rs b/src/connector/src/parser/unified/protobuf.rs index 4bea2cbab306b..02febc22db247 100644 --- a/src/connector/src/parser/unified/protobuf.rs +++ b/src/connector/src/parser/unified/protobuf.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::borrow::Cow; use std::sync::{Arc, LazyLock}; use prost_reflect::{DescriptorPool, DynamicMessage, ReflectMessage}; use risingwave_common::log::LogSuppresser; -use risingwave_common::types::{DataType, DatumCow}; +use risingwave_common::types::{DataType, DatumCow, ToOwnedDatum}; use thiserror_ext::AsReport; use super::{Access, AccessResult}; @@ -56,9 +57,14 @@ impl Access for ProtobufAccess { tracing::error!(suppressed_count, "{}", e.as_report()); } })?; - let value = self.message.get_field(&field_desc); - // TODO: may borrow the value directly - from_protobuf_value(&field_desc, &value, &self.descriptor_pool).map(Into::into) + match self.message.get_field(&field_desc) { + Cow::Borrowed(value) => from_protobuf_value(&field_desc, value, &self.descriptor_pool), + + // `Owned` variant occurs only if there's no such field and the default value is returned. + Cow::Owned(value) => from_protobuf_value(&field_desc, &value, &self.descriptor_pool) + // enforce `Owned` variant to avoid returning a reference to a temporary value + .map(|d| d.to_owned_datum().into()), + } } } From 402212f6259537fa2682880ee4864d17a280d2b6 Mon Sep 17 00:00:00 2001 From: Croxx Date: Tue, 11 Jun 2024 14:30:21 +0800 Subject: [PATCH 07/37] chore(storage): remove unused config (#17175) Signed-off-by: MrCroxx --- src/common/src/config.rs | 14 -------------- src/config/example.toml | 4 ---- src/storage/src/opts.rs | 20 -------------------- 3 files changed, 38 deletions(-) diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 3b6dec9829128..6395cbf37ed15 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -827,12 +827,6 @@ pub struct FileCacheConfig { #[serde(default = "default::file_cache::recover_concurrency")] pub recover_concurrency: usize, - #[serde(default = "default::file_cache::lfu_window_to_cache_size_ratio")] - pub lfu_window_to_cache_size_ratio: usize, - - #[serde(default = "default::file_cache::lfu_tiny_lru_capacity_ratio")] - pub lfu_tiny_lru_capacity_ratio: f64, - #[serde(default = "default::file_cache::insert_rate_limit_mb")] pub insert_rate_limit_mb: usize, @@ -1564,14 +1558,6 @@ pub mod default { 8 } - pub fn lfu_window_to_cache_size_ratio() -> usize { - 1 - } - - pub fn lfu_tiny_lru_capacity_ratio() -> f64 { - 0.01 - } - pub fn insert_rate_limit_mb() -> usize { 0 } diff --git a/src/config/example.toml b/src/config/example.toml index d3d0af3fe6cd1..b35590c85059b 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -165,8 +165,6 @@ file_capacity_mb = 64 flushers = 4 reclaimers = 4 recover_concurrency = 8 -lfu_window_to_cache_size_ratio = 1 -lfu_tiny_lru_capacity_ratio = 0.01 insert_rate_limit_mb = 0 indexer_shards = 64 compression = "none" @@ -178,8 +176,6 @@ file_capacity_mb = 64 flushers = 4 reclaimers = 4 recover_concurrency = 8 -lfu_window_to_cache_size_ratio = 1 -lfu_tiny_lru_capacity_ratio = 0.01 insert_rate_limit_mb = 0 indexer_shards = 64 compression = "none" diff --git a/src/storage/src/opts.rs b/src/storage/src/opts.rs index f553c6ff5ce7a..e9792c5d5e4db 100644 --- a/src/storage/src/opts.rs +++ b/src/storage/src/opts.rs @@ -87,8 +87,6 @@ pub struct StorageOpts { pub data_file_cache_flushers: usize, pub data_file_cache_reclaimers: usize, pub data_file_cache_recover_concurrency: usize, - pub data_file_cache_lfu_window_to_cache_size_ratio: usize, - pub data_file_cache_lfu_tiny_lru_capacity_ratio: f64, pub data_file_cache_insert_rate_limit_mb: usize, pub data_file_cache_indexer_shards: usize, pub data_file_cache_compression: String, @@ -107,8 +105,6 @@ pub struct StorageOpts { pub meta_file_cache_flushers: usize, pub meta_file_cache_reclaimers: usize, pub meta_file_cache_recover_concurrency: usize, - pub meta_file_cache_lfu_window_to_cache_size_ratio: usize, - pub meta_file_cache_lfu_tiny_lru_capacity_ratio: f64, pub meta_file_cache_insert_rate_limit_mb: usize, pub meta_file_cache_indexer_shards: usize, pub meta_file_cache_compression: String, @@ -182,14 +178,6 @@ impl From<(&RwConfig, &SystemParamsReader, &StorageMemoryConfig)> for StorageOpt data_file_cache_flushers: c.storage.data_file_cache.flushers, data_file_cache_reclaimers: c.storage.data_file_cache.reclaimers, data_file_cache_recover_concurrency: c.storage.data_file_cache.recover_concurrency, - data_file_cache_lfu_window_to_cache_size_ratio: c - .storage - .data_file_cache - .lfu_window_to_cache_size_ratio, - data_file_cache_lfu_tiny_lru_capacity_ratio: c - .storage - .data_file_cache - .lfu_tiny_lru_capacity_ratio, data_file_cache_insert_rate_limit_mb: c.storage.data_file_cache.insert_rate_limit_mb, data_file_cache_indexer_shards: c.storage.data_file_cache.indexer_shards, data_file_cache_compression: c.storage.data_file_cache.compression.clone(), @@ -199,14 +187,6 @@ impl From<(&RwConfig, &SystemParamsReader, &StorageMemoryConfig)> for StorageOpt meta_file_cache_flushers: c.storage.meta_file_cache.flushers, meta_file_cache_reclaimers: c.storage.meta_file_cache.reclaimers, meta_file_cache_recover_concurrency: c.storage.meta_file_cache.recover_concurrency, - meta_file_cache_lfu_window_to_cache_size_ratio: c - .storage - .meta_file_cache - .lfu_window_to_cache_size_ratio, - meta_file_cache_lfu_tiny_lru_capacity_ratio: c - .storage - .meta_file_cache - .lfu_tiny_lru_capacity_ratio, meta_file_cache_insert_rate_limit_mb: c.storage.meta_file_cache.insert_rate_limit_mb, meta_file_cache_indexer_shards: c.storage.meta_file_cache.indexer_shards, meta_file_cache_compression: c.storage.meta_file_cache.compression.clone(), From 61bae13dff4aa0f987145a8c1cfcc4e0cdc178aa Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Tue, 11 Jun 2024 14:31:25 +0800 Subject: [PATCH 08/37] fix(sink): do compaction in chunk for upsert sink (#17174) --- src/stream/src/executor/sink.rs | 29 +++++++++++++++++++---------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index 84fde4075a12d..52377c49c322a 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -368,16 +368,25 @@ impl SinkExecutor { // Compact the chunk to eliminate any useless intermediate result (e.g. UPDATE // V->V). let chunk = merge_chunk_row(chunk, &stream_key); - let chunk = if sink_type == SinkType::ForceAppendOnly { - // Force append-only by dropping UPDATE/DELETE messages. We do this when the - // user forces the sink to be append-only while it is actually not based on - // the frontend derivation result. - force_append_only(chunk) - } else { - chunk - }; - - yield Message::Chunk(chunk); + match sink_type { + SinkType::AppendOnly => yield Message::Chunk(chunk), + SinkType::ForceAppendOnly => { + // Force append-only by dropping UPDATE/DELETE messages. We do this when the + // user forces the sink to be append-only while it is actually not based on + // the frontend derivation result. + yield Message::Chunk(force_append_only(chunk)) + } + SinkType::Upsert => { + // Making sure the optimization in https://github.com/risingwavelabs/risingwave/pull/12250 is correct, + // it is needed to do the compaction here. + for chunk in + StreamChunkCompactor::new(stream_key.clone(), vec![chunk]) + .into_compacted_chunks() + { + yield Message::Chunk(chunk) + } + } + } } Message::Barrier(barrier) => { yield Message::Barrier(barrier); From 21babd5e513e3d12bac67a494005d8e8d9b46c68 Mon Sep 17 00:00:00 2001 From: August Date: Tue, 11 Jun 2024 14:47:48 +0800 Subject: [PATCH 09/37] chore: expose associated source in rw_sources (#17150) --- e2e_test/ddl/alter_set_schema.slt | 3 +- e2e_test/source/basic/ddl.slt | 2 +- .../basic/old_row_format_syntax/ddl.slt | 2 +- .../tests/testdata/output/subquery.yaml | 2 +- .../system_catalog/rw_catalog/rw_sources.rs | 67 +++++++++---------- src/frontend/src/handler/show.rs | 1 - 6 files changed, 38 insertions(+), 39 deletions(-) diff --git a/e2e_test/ddl/alter_set_schema.slt b/e2e_test/ddl/alter_set_schema.slt index 9ea95e9cc09c2..74dcc5a77e64a 100644 --- a/e2e_test/ddl/alter_set_schema.slt +++ b/e2e_test/ddl/alter_set_schema.slt @@ -53,13 +53,14 @@ CREATE SOURCE test_source (v INT) WITH ( statement ok ALTER SOURCE test_source SET SCHEMA test_schema; -query TT +query TT rowsort SELECT name AS sourcename, nspname AS schemaname FROM rw_sources JOIN pg_namespace ON pg_namespace.oid = rw_sources.schema_id WHERE nspname = 'test_schema'; ---- test_source test_schema +test_table test_schema statement ok CREATE SINK test_sink AS SELECT u FROM test_schema.test_table WITH ( diff --git a/e2e_test/source/basic/ddl.slt b/e2e_test/source/basic/ddl.slt index 465e0f19344e9..a56d90934c149 100644 --- a/e2e_test/source/basic/ddl.slt +++ b/e2e_test/source/basic/ddl.slt @@ -188,7 +188,7 @@ create table s ( query T show sources ---- - +s query T show tables 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 d5c41d4ded878..b48249ca7e393 100644 --- a/e2e_test/source/basic/old_row_format_syntax/ddl.slt +++ b/e2e_test/source/basic/old_row_format_syntax/ddl.slt @@ -142,7 +142,7 @@ create table s ( query T show sources ---- - +s query T show tables diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index f9ae7cdee3f0d..e113a0aca4d1d 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -241,7 +241,7 @@ │ │ │ │ │ │ │ └─LogicalProject { exprs: [rw_system_tables.id, rw_system_tables.name, 'system table':Varchar, rw_system_tables.schema_id, rw_system_tables.owner, rw_system_tables.definition, rw_system_tables.acl] } │ │ │ │ │ │ │ └─LogicalSysScan { table: rw_system_tables, columns: [rw_system_tables.id, rw_system_tables.name, rw_system_tables.schema_id, rw_system_tables.owner, rw_system_tables.definition, rw_system_tables.acl] } │ │ │ │ │ │ └─LogicalProject { exprs: [rw_sources.id, rw_sources.name, 'source':Varchar, rw_sources.schema_id, rw_sources.owner, rw_sources.definition, rw_sources.acl] } - │ │ │ │ │ │ └─LogicalSysScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id, rw_sources.owner, rw_sources.connector, rw_sources.columns, rw_sources.format, rw_sources.row_encode, rw_sources.append_only, rw_sources.connection_id, rw_sources.definition, rw_sources.acl, rw_sources.initialized_at, rw_sources.created_at, rw_sources.initialized_at_cluster_version, rw_sources.created_at_cluster_version] } + │ │ │ │ │ │ └─LogicalSysScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id, rw_sources.owner, rw_sources.connector, rw_sources.columns, rw_sources.format, rw_sources.row_encode, rw_sources.append_only, rw_sources.associated_table_id, rw_sources.connection_id, rw_sources.definition, rw_sources.acl, rw_sources.initialized_at, rw_sources.created_at, rw_sources.initialized_at_cluster_version, rw_sources.created_at_cluster_version] } │ │ │ │ │ └─LogicalProject { exprs: [rw_indexes.id, rw_indexes.name, 'index':Varchar, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl] } │ │ │ │ │ └─LogicalSysScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.primary_table_id, rw_indexes.key_columns, rw_indexes.include_columns, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl, rw_indexes.initialized_at, rw_indexes.created_at, rw_indexes.initialized_at_cluster_version, rw_indexes.created_at_cluster_version] } │ │ │ │ └─LogicalProject { exprs: [rw_sinks.id, rw_sinks.name, 'sink':Varchar, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.definition, rw_sinks.acl] } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs index 5a05f69750caf..441ac99b48cda 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs @@ -32,6 +32,7 @@ struct RwSource { format: Option, row_encode: Option, append_only: bool, + associated_table_id: Option, connection_id: Option, definition: String, acl: String, @@ -51,40 +52,38 @@ fn read_rw_sources_info(reader: &SysCatalogReaderImpl) -> Result> Ok(schemas .flat_map(|schema| { - schema - .iter_source() - .filter(|s| s.associated_table_id.is_none()) - .map(|source| RwSource { - id: source.id as i32, - name: source.name.clone(), - schema_id: schema.id() as i32, - owner: source.owner as i32, - connector: source - .with_properties - .get(UPSTREAM_SOURCE_KEY) - .cloned() - .unwrap_or("".to_string()) - .to_uppercase(), - columns: source.columns.iter().map(|c| c.name().into()).collect(), - format: source - .info - .get_format() - .ok() - .map(|format| format.as_str_name().into()), - row_encode: source - .info - .get_row_encode() - .ok() - .map(|row_encode| row_encode.as_str_name().into()), - append_only: source.append_only, - connection_id: source.connection_id.map(|id| id as i32), - definition: source.create_sql(), - acl: get_acl_items(&Object::SourceId(source.id), false, &users, username_map), - initialized_at: source.initialized_at_epoch.map(|e| e.as_timestamptz()), - created_at: source.created_at_epoch.map(|e| e.as_timestamptz()), - initialized_at_cluster_version: source.initialized_at_cluster_version.clone(), - created_at_cluster_version: source.created_at_cluster_version.clone(), - }) + schema.iter_source().map(|source| RwSource { + id: source.id as i32, + name: source.name.clone(), + schema_id: schema.id() as i32, + owner: source.owner as i32, + connector: source + .with_properties + .get(UPSTREAM_SOURCE_KEY) + .cloned() + .unwrap_or("".to_string()) + .to_uppercase(), + columns: source.columns.iter().map(|c| c.name().into()).collect(), + format: source + .info + .get_format() + .ok() + .map(|format| format.as_str_name().into()), + row_encode: source + .info + .get_row_encode() + .ok() + .map(|row_encode| row_encode.as_str_name().into()), + append_only: source.append_only, + associated_table_id: source.associated_table_id.map(|id| id.table_id as i32), + connection_id: source.connection_id.map(|id| id as i32), + definition: source.create_sql(), + acl: get_acl_items(&Object::SourceId(source.id), false, &users, username_map), + initialized_at: source.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: source.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: source.initialized_at_cluster_version.clone(), + created_at_cluster_version: source.created_at_cluster_version.clone(), + }) }) .collect()) } diff --git a/src/frontend/src/handler/show.rs b/src/frontend/src/handler/show.rs index f2d5186b67962..b93a8032cbcd4 100644 --- a/src/frontend/src/handler/show.rs +++ b/src/frontend/src/handler/show.rs @@ -318,7 +318,6 @@ pub async fn handle_show_object( .read_guard() .get_schema_by_name(session.database(), &schema_or_default(&schema))? .iter_source() - .filter(|t| t.associated_table_id.is_none()) .map(|t| t.name.clone()) .collect(), ShowObject::Sink { schema } => catalog_reader From e6b8e1283de6fd5ead7672ccb2980ff01ec7adf4 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 11 Jun 2024 15:36:23 +0800 Subject: [PATCH 10/37] doc(stream): add docs on initializing the replicated state table (#17027) --- docs/backfill.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/docs/backfill.md b/docs/backfill.md index 6784070853712..aac20615caf10 100644 --- a/docs/backfill.md +++ b/docs/backfill.md @@ -358,6 +358,22 @@ and arrangement backfill will consume this historical data snapshot: | 1 | 'Jack' | 29 | | 2 | 'Jill' | 30 | +#### Initialization + +Something to note is that for the first snapshot, +upstream may not have finished committing data in that epoch to s3. + +Additionally, we have not replicated any upstream records +during that epoch, only in the subsequent ones. + +As such, we must wait for that first checkpoint to be committed, +before reading, or we risk missing the uncommitted data in our backfill. + +This is supported internally inside `init_epoch` for replicated state table. +```shell + upstream_table.init_epoch(first_epoch).await?; +``` + ### Recovery TODO From daa1c42c9966978de570968a7f5b56523050a7b1 Mon Sep 17 00:00:00 2001 From: Dylan Date: Tue, 11 Jun 2024 18:54:12 +0800 Subject: [PATCH 11/37] feat: calculate reserved memory based on a gradient proportion (#16992) --- src/compute/src/memory/config.rs | 52 ++++++++++++++++++++++++++++---- 1 file changed, 46 insertions(+), 6 deletions(-) diff --git a/src/compute/src/memory/config.rs b/src/compute/src/memory/config.rs index 2a908d850f3f1..2a6607c268a67 100644 --- a/src/compute/src/memory/config.rs +++ b/src/compute/src/memory/config.rs @@ -27,7 +27,9 @@ pub const MIN_COMPUTE_MEMORY_MB: usize = 512; /// overhead, network buffer, etc.) in megabytes. pub const MIN_SYSTEM_RESERVED_MEMORY_MB: usize = 512; -const SYSTEM_RESERVED_MEMORY_PROPORTION: f64 = 0.3; +const RESERVED_MEMORY_LEVELS: [usize; 2] = [16 << 30, usize::MAX]; + +const RESERVED_MEMORY_PROPORTIONS: [f64; 2] = [0.3, 0.2]; const STORAGE_MEMORY_PROPORTION: f64 = 0.3; @@ -44,7 +46,7 @@ const STORAGE_SHARED_BUFFER_MEMORY_PROPORTION: f64 = 0.3; const COMPUTE_BATCH_MEMORY_PROPORTION: f64 = 0.3; /// Each compute node reserves some memory for stack and code segment of processes, allocation -/// overhead, network buffer, etc. based on `SYSTEM_RESERVED_MEMORY_PROPORTION`. The reserve memory +/// overhead, network buffer, etc. based on gradient reserve memory proportion. The reserve memory /// size must be larger than `MIN_SYSTEM_RESERVED_MEMORY_MB` pub fn reserve_memory_bytes(opts: &ComputeNodeOpts) -> (usize, usize) { if opts.total_memory_bytes < MIN_COMPUTE_MEMORY_MB << 20 { @@ -55,10 +57,10 @@ pub fn reserve_memory_bytes(opts: &ComputeNodeOpts) -> (usize, usize) { ); } - // If `reserved_memory_bytes` is not set, use `SYSTEM_RESERVED_MEMORY_PROPORTION` * `total_memory_bytes`. - let reserved = opts.reserved_memory_bytes.unwrap_or_else(|| { - (opts.total_memory_bytes as f64 * SYSTEM_RESERVED_MEMORY_PROPORTION).ceil() as usize - }); + // If `reserved_memory_bytes` is not set, calculate total_memory_bytes based on gradient reserve memory proportion. + let reserved = opts + .reserved_memory_bytes + .unwrap_or_else(|| gradient_reserve_memory_bytes(opts.total_memory_bytes)); // Should have at least `MIN_SYSTEM_RESERVED_MEMORY_MB` for reserved memory. let reserved = std::cmp::max(reserved, MIN_SYSTEM_RESERVED_MEMORY_MB << 20); @@ -66,6 +68,31 @@ pub fn reserve_memory_bytes(opts: &ComputeNodeOpts) -> (usize, usize) { (reserved, opts.total_memory_bytes - reserved) } +/// Calculate the reserved memory based on the total memory size. +/// The reserved memory size is calculated based on the following gradient: +/// - 30% of the first 16GB +/// - 20% of the rest +fn gradient_reserve_memory_bytes(total_memory_bytes: usize) -> usize { + let mut total_memory_bytes = total_memory_bytes; + let mut reserved = 0; + for i in 0..RESERVED_MEMORY_LEVELS.len() { + let level_diff = if i == 0 { + RESERVED_MEMORY_LEVELS[0] + } else { + RESERVED_MEMORY_LEVELS[i] - RESERVED_MEMORY_LEVELS[i - 1] + }; + if total_memory_bytes <= level_diff { + reserved += (total_memory_bytes as f64 * RESERVED_MEMORY_PROPORTIONS[i]) as usize; + break; + } else { + reserved += (level_diff as f64 * RESERVED_MEMORY_PROPORTIONS[i]) as usize; + total_memory_bytes -= level_diff; + } + } + + reserved +} + /// Decide the memory limit for each storage cache. If not specified in `StorageConfig`, memory /// limits are calculated based on the proportions to total `non_reserved_memory_bytes`. pub fn storage_memory_config( @@ -346,4 +373,17 @@ mod tests { assert_eq!(memory_config.shared_buffer_capacity_mb, 1024); assert_eq!(memory_config.compactor_memory_limit_mb, 512); } + + #[test] + fn test_gradient_reserve_memory_bytes() { + assert_eq!(super::gradient_reserve_memory_bytes(4 << 30), 1288490188); + assert_eq!(super::gradient_reserve_memory_bytes(8 << 30), 2576980377); + assert_eq!(super::gradient_reserve_memory_bytes(16 << 30), 5153960755); + assert_eq!(super::gradient_reserve_memory_bytes(24 << 30), 6871947673); + assert_eq!(super::gradient_reserve_memory_bytes(32 << 30), 8589934591); + assert_eq!(super::gradient_reserve_memory_bytes(54 << 30), 13314398617); + assert_eq!(super::gradient_reserve_memory_bytes(64 << 30), 15461882265); + assert_eq!(super::gradient_reserve_memory_bytes(100 << 30), 23192823398); + assert_eq!(super::gradient_reserve_memory_bytes(128 << 30), 29205777612); + } } From e9201ac42365de352a62c7554a70c739056e2030 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 11 Jun 2024 23:06:52 +0800 Subject: [PATCH 12/37] fix(ci): increase sleep duration before restarting cluster in backfill sink test (#17183) --- ci/scripts/run-backfill-tests.sh | 3 +++ e2e_test/backfill/sink/create_sink.slt | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/ci/scripts/run-backfill-tests.sh b/ci/scripts/run-backfill-tests.sh index ac552cfcdcdd0..c03762a7150a7 100755 --- a/ci/scripts/run-backfill-tests.sh +++ b/ci/scripts/run-backfill-tests.sh @@ -185,6 +185,9 @@ test_sink_backfill_recovery() { # Check progress sqllogictest -p 4566 -d dev 'e2e_test/backfill/sink/create_sink.slt' + # Sleep before restart cluster, to ensure the downstream sink actually gets created. + sleep 5 + # Restart restart_cluster sleep 5 diff --git a/e2e_test/backfill/sink/create_sink.slt b/e2e_test/backfill/sink/create_sink.slt index 017eb8e693de2..016e3bcb2049b 100644 --- a/e2e_test/backfill/sink/create_sink.slt +++ b/e2e_test/backfill/sink/create_sink.slt @@ -5,9 +5,9 @@ statement ok create table t (v1 int); statement ok -SET STREAMING_RATE_LIMIT = 1000; +SET STREAMING_RATE_LIMIT = 500; -# Should finish in 10s +# Should finish in 20s statement ok insert into t select * from generate_series(1, 10000); From bdf42d86996597e07ecca6acbfa451b8df9bcc02 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 11 Jun 2024 23:09:41 +0800 Subject: [PATCH 13/37] test(backfill): adjust the measured progress bounds for backfill test (#17181) --- .../simulation/tests/integration_tests/backfill_tests.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/tests/simulation/tests/integration_tests/backfill_tests.rs b/src/tests/simulation/tests/integration_tests/backfill_tests.rs index ba34f45e6af67..1d8f7bb727101 100644 --- a/src/tests/simulation/tests/integration_tests/backfill_tests.rs +++ b/src/tests/simulation/tests/integration_tests/backfill_tests.rs @@ -256,7 +256,8 @@ async fn test_arrangement_backfill_progress() -> Result<()> { .run("CREATE MATERIALIZED VIEW m1 AS SELECT * FROM t") .await?; - // Verify arrangement backfill progress after 10s, it should be 1% at least. + // Verify arrangement backfill progress after 10s, it should be around 1%, + // since 10s = 10 records processed. sleep(Duration::from_secs(10)).await; let progress = session .run("SELECT progress FROM rw_catalog.rw_ddl_progress") @@ -264,7 +265,7 @@ async fn test_arrangement_backfill_progress() -> Result<()> { let progress = progress.replace('%', ""); let progress = progress.parse::().unwrap(); assert!( - (1.0..10.0).contains(&progress), + (0.5..1.5).contains(&progress), "progress not within bounds {}", progress ); From 531aa8bffeb8e515bd89290a6ee70e1792419e24 Mon Sep 17 00:00:00 2001 From: alicelol Date: Tue, 11 Jun 2024 09:52:22 -0700 Subject: [PATCH 14/37] doc: update README.md to notice the user about RisingWave's usage of Scarf (#17012) Co-authored-by: hengm3467 <100685635+hengm3467@users.noreply.github.com> --- README.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/README.md b/README.md index 47893c7bea1ba..e54bcf835082c 100644 --- a/README.md +++ b/README.md @@ -71,6 +71,9 @@ Then follow the prompts to start and connect to RisingWave. To learn about other installation options, such as using a Docker image, see [Quick Start](https://docs.risingwave.com/docs/current/get-started/). +> Please note: RisingWave uses [Scarf](https://scarf.sh/) to collect anonymized installation analytics. These analytics help support us understand and improve the distribution of our package. +> The privacy policy of Scarf is available at [https://about.scarf.sh/privacy-policy](https://about.scarf.sh/privacy-policy). + ## Production deployments [**RisingWave Cloud**](https://cloud.risingwave.com) offers the easiest way to run RisingWave in production, with a _forever-free_ developer tier. From 36a649f1ccb6464621f04936a715745ea10cf253 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed, 12 Jun 2024 13:21:52 +0800 Subject: [PATCH 15/37] chore: fix README trailing space ci failure (#17212) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index e54bcf835082c..156b53060313d 100644 --- a/README.md +++ b/README.md @@ -71,7 +71,7 @@ Then follow the prompts to start and connect to RisingWave. To learn about other installation options, such as using a Docker image, see [Quick Start](https://docs.risingwave.com/docs/current/get-started/). -> Please note: RisingWave uses [Scarf](https://scarf.sh/) to collect anonymized installation analytics. These analytics help support us understand and improve the distribution of our package. +> Please note: RisingWave uses [Scarf](https://scarf.sh/) to collect anonymized installation analytics. These analytics help support us understand and improve the distribution of our package. > The privacy policy of Scarf is available at [https://about.scarf.sh/privacy-policy](https://about.scarf.sh/privacy-policy). ## Production deployments From 6d96f4b5a818e64bc0549471e629b5dcb8d6d82f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Jun 2024 13:23:05 +0800 Subject: [PATCH 16/37] chore(deps-dev): Bump braces from 3.0.2 to 3.0.3 in /dashboard (#17194) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- dashboard/package-lock.json | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/dashboard/package-lock.json b/dashboard/package-lock.json index 5bf9ae127252e..857888e4d4cfe 100644 --- a/dashboard/package-lock.json +++ b/dashboard/package-lock.json @@ -3870,12 +3870,12 @@ } }, "node_modules/braces": { - "version": "3.0.2", - "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.2.tgz", - "integrity": "sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A==", + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.3.tgz", + "integrity": "sha512-yQbXgO/OSZVD2IsiLlro+7Hf6Q18EJrKSEsdoMzKePKXct3gvD8oLcOQdIzGupr5Fj+EDe8gO/lxc1BzfMpxvA==", "dev": true, "dependencies": { - "fill-range": "^7.0.1" + "fill-range": "^7.1.1" }, "engines": { "node": ">=8" @@ -6325,9 +6325,9 @@ } }, "node_modules/fill-range": { - "version": "7.0.1", - "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.0.1.tgz", - "integrity": "sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ==", + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.1.1.tgz", + "integrity": "sha512-YsGpe3WHLK8ZYi4tWDg2Jy3ebRz2rXowDxnld4bkQB00cc/1Zw9AWnC0i9ztDJitivtQvaI9KaLyKrc+hBW0yg==", "dev": true, "dependencies": { "to-regex-range": "^5.0.1" @@ -14516,12 +14516,12 @@ } }, "braces": { - "version": "3.0.2", - "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.2.tgz", - "integrity": "sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A==", + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.3.tgz", + "integrity": "sha512-yQbXgO/OSZVD2IsiLlro+7Hf6Q18EJrKSEsdoMzKePKXct3gvD8oLcOQdIzGupr5Fj+EDe8gO/lxc1BzfMpxvA==", "dev": true, "requires": { - "fill-range": "^7.0.1" + "fill-range": "^7.1.1" } }, "browser-process-hrtime": { @@ -16350,9 +16350,9 @@ } }, "fill-range": { - "version": "7.0.1", - "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.0.1.tgz", - "integrity": "sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ==", + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.1.1.tgz", + "integrity": "sha512-YsGpe3WHLK8ZYi4tWDg2Jy3ebRz2rXowDxnld4bkQB00cc/1Zw9AWnC0i9ztDJitivtQvaI9KaLyKrc+hBW0yg==", "dev": true, "requires": { "to-regex-range": "^5.0.1" From bb48904320b656c6670d29ef404892396d7b7ac8 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 12 Jun 2024 14:16:18 +0800 Subject: [PATCH 17/37] chore(sink): improve force append only warning (#17213) Signed-off-by: Richard Chien --- src/frontend/src/optimizer/plan_node/stream_sink.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 9c76bfdece25d..49378ab0a53ed 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -470,7 +470,11 @@ impl StreamSink { (false, true, false) => { Err(ErrorCode::SinkError(Box::new(Error::new( ErrorKind::InvalidInput, - format!("The sink cannot be append-only. Please add \"force_append_only='true'\" in {} options to force the sink to be append-only. Notice that this will cause the sink executor to drop any UPDATE or DELETE message.", if syntax_legacy { "WITH" } else { "FORMAT ENCODE" }), + format!( + "The sink cannot be append-only. Please add \"force_append_only='true'\" in {} options to force the sink to be append-only. \ + Notice that this will cause the sink executor to drop DELETE messages and convert UPDATE messages to INSERT.", + if syntax_legacy { "WITH" } else { "FORMAT ENCODE" } + ), ))) .into()) } From 617ed5a8777e108bdcde4fab85cb53668b35303b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 12 Jun 2024 14:37:44 +0800 Subject: [PATCH 18/37] refactor(cdc): refine error report when fail to derive table schema (#17210) Co-authored-by: Bugen Zhao --- src/frontend/src/handler/create_table.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index edeb4ae5441d6..0f3693653ced5 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -1094,7 +1094,9 @@ async fn derive_schema_for_cdc_table( let config = ExternalTableConfig::try_from_btreemap(connect_properties) .context("failed to extract external table config")?; - let table = ExternalTableImpl::connect(config).await?; + let table = ExternalTableImpl::connect(config) + .await + .context("failed to auto derive table schema")?; Ok(( table .column_descs() From cf28b769b5587d5e1367c4f569c91e24712b04fe Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 12 Jun 2024 14:39:25 +0800 Subject: [PATCH 19/37] chore(ci): rework ci labels used to trigger workflows (#17197) --- ci/workflows/main-cron.yml | 104 ++++++++++++++-------------------- ci/workflows/pull-request.yml | 50 ++++++---------- docs/developer-guide.md | 25 ++++---- 3 files changed, 74 insertions(+), 105 deletions(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 14f3a23161c80..7a1054a0d481b 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -7,10 +7,6 @@ auto-retry: &auto-retry steps: - label: "build" command: "ci/scripts/build.sh -p ci-release" - if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-build" - || build.env("CI_STEPS") =~ /(^|,)build(,|$$)/ key: "build" plugins: - docker-compose#v5.1.0: @@ -22,10 +18,6 @@ steps: - label: "build other components" command: "ci/scripts/build-other.sh" - if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-build-other" - || build.env("CI_STEPS") =~ /(^|,)build-other(,|$$)/ key: "build-other" plugins: - seek-oss/aws-sm#v2.3.1: @@ -42,10 +34,6 @@ steps: - label: "build simulation test" command: "ci/scripts/build-simulation.sh" - if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-build-simulation" - || build.env("CI_STEPS") =~ /(^|,)build-simulation(,|$$)/ key: "build-simulation" plugins: - docker-compose#v5.1.0: @@ -57,10 +45,6 @@ steps: - label: "docslt" command: "ci/scripts/docslt.sh" - if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-docslt" - || build.env("CI_STEPS") =~ /(^|,)docslt(,|$$)/ key: "docslt" plugins: - docker-compose#v5.1.0: @@ -74,7 +58,7 @@ steps: key: "e2e-test-release" command: "ci/scripts/cron-e2e-test.sh -p ci-release -m ci-3streaming-2serving-3fe" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-test" || build.env("CI_STEPS") =~ /(^|,)e2e-tests?(,|$$)/ depends_on: @@ -94,7 +78,7 @@ steps: key: "slow-e2e-test-release" command: "ci/scripts/slow-e2e-test.sh -p ci-release -m ci-3streaming-2serving-3fe" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-slow-e2e-tests" || build.env("CI_STEPS") =~ /(^|,)slow-e2e-tests?(,|$$)/ depends_on: @@ -113,7 +97,7 @@ steps: key: "e2e-meta-backup-test-release" command: "ci/scripts/run-meta-backup-test.sh -p ci-release -m ci-3streaming-2serving-3fe" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-meta-backup-test" || build.env("CI_STEPS") =~ /(^|,)e2e-tests?(,|$$)/ depends_on: @@ -133,7 +117,7 @@ steps: key: "e2e-test-release-parallel" command: "ci/scripts/e2e-test-parallel.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-parallel-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-parallel-tests?(,|$$)/ depends_on: @@ -158,7 +142,7 @@ steps: key: "e2e-test-release-parallel-memory" command: "ci/scripts/e2e-test-parallel-in-memory.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-parallel-in-memory-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-parallel-in-memory-tests?(,|$$)/ depends_on: @@ -177,7 +161,7 @@ steps: key: "e2e-test-release-source" command: "ci/scripts/e2e-source-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-source-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-source-tests?(,|$$)/ depends_on: @@ -196,7 +180,7 @@ steps: key: "e2e-test-release-sink" command: "ci/scripts/e2e-sink-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-sink-tests?(,|$$)/ depends_on: @@ -215,7 +199,7 @@ steps: key: "fuzz-test" command: "ci/scripts/cron-fuzz-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-sqlsmith-fuzzing-tests" || build.env("CI_STEPS") =~ /(^|,)sqlsmith-fuzzing-tests?(,|$$)/ depends_on: @@ -237,7 +221,7 @@ steps: key: "unit-test" command: "ci/scripts/unit-test.sh" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-unit-test" || build.env("CI_STEPS") =~ /(^|,)unit-tests?(,|$$)/ plugins: @@ -257,7 +241,7 @@ steps: key: "unit-test-deterministic" command: "MADSIM_TEST_NUM=100 timeout 30m ci/scripts/deterministic-unit-test.sh" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-unit-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)unit-tests?-deterministic-simulation(,|$$)/ plugins: @@ -272,7 +256,7 @@ steps: key: "integration-test-deterministic-scale" command: "TEST_NUM=60 ci/scripts/deterministic-it-test.sh scale::" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)integration-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -289,7 +273,7 @@ steps: key: "integration-test-deterministic-recovery" command: "TEST_NUM=60 ci/scripts/deterministic-it-test.sh recovery::" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)integration-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -306,7 +290,7 @@ steps: key: "integration-test-deterministic-backfill" command: "TEST_NUM=30 ci/scripts/deterministic-it-test.sh backfill_tests::" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)integration-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -323,7 +307,7 @@ steps: key: "integration-test-deterministic-storage" command: "TEST_NUM=30 ci/scripts/deterministic-it-test.sh storage::" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)integration-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -340,7 +324,7 @@ steps: key: "integration-test-deterministic-sink" command: "TEST_NUM=30 ci/scripts/deterministic-it-test.sh sink::" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)integration-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -357,7 +341,7 @@ steps: key: "e2e-test-deterministic" command: "TEST_NUM=64 timeout 75m ci/scripts/deterministic-e2e-test.sh" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)e2e-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -379,7 +363,7 @@ steps: key: "recovery-test-deterministic" command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.0 timeout 65m ci/scripts/deterministic-recovery-test.sh" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)recovery-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -398,7 +382,7 @@ steps: key: "background-ddl-arrangement-backfill-recovery-test-deterministic" command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.8 USE_ARRANGEMENT_BACKFILL=--use-arrangement-backfill timeout 65m ci/scripts/deterministic-recovery-test.sh" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)recovery-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -416,7 +400,7 @@ steps: key: "e2e-iceberg-sink-test" command: "ci/scripts/e2e-iceberg-sink-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-iceberg-sink-tests?(,|$$)/ depends_on: @@ -436,7 +420,7 @@ steps: key: "e2e-iceberg-sink-v2-test" command: "ci/scripts/e2e-iceberg-sink-v2-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-iceberg-sink-v2-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-iceberg-sink-v2-tests?(,|$$)/ depends_on: @@ -455,7 +439,7 @@ steps: key: "e2e-java-binding-tests" command: "ci/scripts/java-binding-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-java-binding-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-java-binding-tests?(,|$$)/ depends_on: @@ -476,7 +460,7 @@ steps: key: "s3-v2-source-check-aws-json-parser" command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_v2.py -t json" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-s3-source-tests" || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ depends_on: build @@ -498,7 +482,7 @@ steps: key: "s3-v2-source-batch-read-check-aws-json-parser" command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_batch.py -t json" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-s3-source-tests" || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ depends_on: build @@ -520,7 +504,7 @@ steps: key: "s3-v2-source-check-aws-csv-parser" command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_v2.py -t csv_without_header" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-s3-source-tests" || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ depends_on: build @@ -542,7 +526,7 @@ steps: key: "s3-source-test-for-opendal-fs-engine-csv-parser" command: "ci/scripts/s3-source-test.sh -p ci-release -s posix_fs_source.py -t csv_without_header" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-s3-source-tests" || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ depends_on: build @@ -560,7 +544,7 @@ steps: # key: "s3-source-test-for-opendal-fs-engine" # command: "ci/scripts/s3-source-test-for-opendal-fs-engine.sh -p ci-release -s gcs_source.py" # if: | - # !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + # !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null # || build.pull_request.labels includes "ci/run-s3-source-tests" # || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ # depends_on: build @@ -582,7 +566,7 @@ steps: key: "pulsar-source-tests" command: "ci/scripts/pulsar-source-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-pulsar-source-tests" || build.env("CI_STEPS") =~ /(^|,)pulsar-source-tests?(,|$$)/ depends_on: @@ -607,7 +591,7 @@ steps: key: "run-micro-benchmarks" command: "ci/scripts/run-micro-benchmarks.sh" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-micro-benchmarks" || build.env("CI_STEPS") =~ /(^|,)micro-benchmarks?(,|$$)/ plugins: @@ -622,7 +606,7 @@ steps: key: "upload-micro-benchmarks" if: | build.branch == "main" - || !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-micro-benchmarks" || build.env("CI_STEPS") =~ /(^|,)micro-benchmarks?(,|$$)/ command: @@ -647,7 +631,7 @@ steps: key: "backwards-compat-tests" command: "VERSION_OFFSET={{matrix.version_offset}} RW_COMMIT=$BUILDKITE_COMMIT ci/scripts/backwards-compat-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-backwards-compat-tests" || build.env("CI_STEPS") =~ /(^|,)backwards?-compat-tests?(,|$$)/ depends_on: @@ -681,7 +665,7 @@ steps: key: "sqlsmith-differential-tests" command: "ci/scripts/sqlsmith-differential-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-sqlsmith-differential-tests" || build.env("CI_STEPS") =~ /(^|,)sqlsmith-differential-tests?(,|$$)/ depends_on: @@ -697,7 +681,7 @@ steps: key: "backfill-tests" command: "BUILDKITE=${BUILDKITE:-} ci/scripts/backfill-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-backfill-tests" || build.env("CI_STEPS") =~ /(^|,)backfill-tests?(,|$$)/ depends_on: @@ -715,7 +699,7 @@ steps: key: "e2e-standalone-binary-tests" command: "ci/scripts/e2e-test.sh -p ci-release -m standalone" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-standalone-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-standalone-tests?(,|$$)/ depends_on: @@ -735,7 +719,7 @@ steps: key: "e2e-single-node-binary-tests" command: "ci/scripts/e2e-test.sh -p ci-release -m single-node" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-single-node-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-single-node-tests?(,|$$)/ depends_on: @@ -755,7 +739,7 @@ steps: key: "e2e-test-opendal-parallel" command: "ci/scripts/e2e-test-parallel-for-opendal.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-parallel-tests-for-opendal" || build.env("CI_STEPS") =~ /(^|,)e2e-parallel-tests?-for-opendal(,|$$)/ depends_on: @@ -774,7 +758,7 @@ steps: key: "e2e-deltalake-sink-rust-tests" command: "ci/scripts/e2e-deltalake-sink-rust-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-deltalake-sink-rust-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-deltalake-sink-rust-tests?(,|$$)/ depends_on: @@ -793,7 +777,7 @@ steps: key: "e2e-redis-sink-tests" command: "ci/scripts/e2e-redis-sink-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-redis-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-redis-sink-tests?(,|$$)/ depends_on: @@ -812,7 +796,7 @@ steps: key: "e2e-doris-sink-tests" command: "ci/scripts/e2e-doris-sink-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-doris-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-doris-sink-tests?(,|$$)/ depends_on: @@ -831,7 +815,7 @@ steps: key: "e2e-starrocks-sink-tests" command: "ci/scripts/e2e-starrocks-sink-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-starrocks-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-starrocks-sink-tests?(,|$$)/ depends_on: @@ -850,7 +834,7 @@ steps: key: "e2e-cassandra-sink-tests" command: "ci/scripts/e2e-cassandra-sink-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-cassandra-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-cassandra-sink-tests?(,|$$)/ depends_on: @@ -869,7 +853,7 @@ steps: key: "e2e-clickhouse-sink-tests" command: "ci/scripts/e2e-clickhouse-sink-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-clickhouse-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-clickhouse-sink-tests?(,|$$)/ depends_on: @@ -888,7 +872,7 @@ steps: key: "e2e-pulsar-sink-tests" command: "ci/scripts/e2e-pulsar-sink-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-pulsar-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-pulsar-sink-tests?(,|$$)/ depends_on: @@ -907,7 +891,7 @@ steps: key: "e2e-mqtt-sink-tests" command: "ci/scripts/e2e-mqtt-sink-test.sh -p ci-release" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-mqtt-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-mqtt-sink-tests?(,|$$)/ depends_on: @@ -926,7 +910,7 @@ steps: key: "connector-node-integration-test" command: "ci/scripts/connector-node-integration-test.sh -p ci-release -v {{matrix.java_version}}" if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-connector-node-integration-tests" || build.env("CI_STEPS") =~ /(^|,)connector-node-integration-tests?(,|$$)/ depends_on: diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 3f0c307bdab30..385790e830232 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -34,10 +34,6 @@ steps: - label: "build" command: "ci/scripts/build.sh -p ci-dev" key: "build" - if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-build" - || build.env("CI_STEPS") =~ /(^|,)build(,|$$)/ plugins: - gencer/cache#v2.4.10: *cargo-cache - docker-compose#v5.1.0: @@ -50,10 +46,6 @@ steps: - label: "build other components" command: "ci/scripts/build-other.sh" key: "build-other" - if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-build-other" - || build.env("CI_STEPS") =~ /(^|,)build-other(,|$$)/ plugins: - gencer/cache#v2.4.10: *cargo-cache - seek-oss/aws-sm#v2.3.1: @@ -71,10 +63,6 @@ steps: - label: "build (deterministic simulation)" command: "ci/scripts/build-simulation.sh" key: "build-simulation" - if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-build-simulation" - || build.env("CI_STEPS") =~ /(^|,)build-simulation(,|$$)/ plugins: - gencer/cache#v2.4.10: *cargo-cache - docker-compose#v5.1.0: @@ -86,10 +74,6 @@ steps: - label: "docslt" command: "ci/scripts/docslt.sh" key: "docslt" - if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-docslt" - || build.env("CI_STEPS") =~ /(^|,)docslt(,|$$)/ plugins: - gencer/cache#v2.4.10: *cargo-cache - docker-compose#v5.1.0: @@ -102,7 +86,7 @@ steps: - label: "end-to-end test" command: "ci/scripts/e2e-test.sh -p ci-dev -m ci-3streaming-2serving-3fe" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-test" || build.env("CI_STEPS") =~ /(^|,)e2e-tests?(,|$$)/ depends_on: @@ -122,7 +106,7 @@ steps: key: "slow-e2e-test" command: "ci/scripts/slow-e2e-test.sh -p ci-dev -m ci-3streaming-2serving-3fe" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-slow-e2e-tests" || build.env("CI_STEPS") =~ /(^|,)slow-e2e-tests?(,|$$)/ depends_on: @@ -140,7 +124,7 @@ steps: - label: "end-to-end test (parallel)" command: "ci/scripts/e2e-test-parallel.sh -p ci-dev" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-parallel-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-parallel-tests?(,|$$)/ depends_on: @@ -186,7 +170,7 @@ steps: - label: "end-to-end source test" command: "ci/scripts/e2e-source-test.sh -p ci-dev" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-source-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-source-tests?(,|$$)/ depends_on: @@ -205,7 +189,7 @@ steps: - label: "end-to-end sink test" command: "ci/scripts/e2e-sink-test.sh -p ci-dev" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-sink-tests?(,|$$)/ depends_on: @@ -424,7 +408,7 @@ steps: - label: "regress test" command: "ci/scripts/regress-test.sh -p ci-dev" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-regress-test" || build.env("CI_STEPS") =~ /(^|,)regress-tests?(,|$$)/ depends_on: "build" @@ -443,7 +427,7 @@ steps: - label: "unit test" command: "ci/scripts/pr-unit-test.sh" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-unit-test" || build.env("CI_STEPS") =~ /(^|,)unit-tests?(,|$$)/ plugins: @@ -463,7 +447,7 @@ steps: - label: "check" command: "ci/scripts/check.sh" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-check" || build.env("CI_STEPS") =~ /(^|,)check(,|$$)/ plugins: @@ -477,7 +461,7 @@ steps: - label: "check dylint" command: "ci/scripts/check-dylint.sh" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-check" || build.env("CI_STEPS") =~ /(^|,)check(,|$$)/ plugins: @@ -491,7 +475,7 @@ steps: - label: "unit test (deterministic simulation)" command: "ci/scripts/deterministic-unit-test.sh" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-unit-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)unit-tests?-deterministic-simulation(,|$$)/ plugins: @@ -506,7 +490,7 @@ steps: - label: "integration test (deterministic simulation)" command: "TEST_NUM=5 ci/scripts/deterministic-it-test.sh" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)integration-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -522,7 +506,7 @@ steps: - label: "end-to-end test (deterministic simulation)" command: "TEST_NUM=16 ci/scripts/deterministic-e2e-test.sh" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)e2e-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -544,7 +528,7 @@ steps: - label: "recovery test (deterministic simulation)" command: "TEST_NUM=8 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.0 ci/scripts/deterministic-recovery-test.sh" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation" || build.env("CI_STEPS") =~ /(^|,)recovery-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" @@ -568,7 +552,7 @@ steps: - label: "misc check" command: "ci/scripts/misc-check.sh" if: | - !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + !(build.pull_request.labels includes "ci/pr/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-misc-check" || build.env("CI_STEPS") =~ /(^|,)misc-check(,|$$)/ plugins: @@ -786,10 +770,10 @@ steps: timeout_in_minutes: 15 retry: *auto-retry - - label: "enable ci/skip-ci only in draft PRs" - if: build.pull_request.labels includes "ci/skip-ci" && !build.pull_request.draft + - label: "enable ci/pr/run-selected only in draft PRs" + if: build.pull_request.labels includes "ci/pr/run-selected" && !build.pull_request.draft commands: - - echo "ci/skip-ci is only usable for draft Pull Requests" + - echo "ci/pr/run-selected is only usable for draft Pull Requests" - exit 1 - label: "micro benchmark" diff --git a/docs/developer-guide.md b/docs/developer-guide.md index 054d33d4d7270..9b03c60f3d9a1 100644 --- a/docs/developer-guide.md +++ b/docs/developer-guide.md @@ -548,17 +548,18 @@ Instructions about submitting PRs are included in the [contribution guidelines]( ## CI Labels Guide -- `[ci/run-xxx ...]`: Run additional steps indicated by `ci/run-xxx` in your PR. -- `ci/skip-ci` + `[ci/run-xxx ...]` : Skip steps except for those indicated by `ci/run-xxx` in your **DRAFT PR.** -- `ci/run-main-cron`: Run full `main-cron`. -- `ci/run-main-cron` + `ci/main-cron/skip-ci` + `[ci/run-xxx …]` : Run specific steps indicated by `ci/run-xxx` +- `[ci/run-xxx ...]`: Run additional steps in the PR workflow indicated by `ci/run-xxx` in your PR. +- `ci/pr/run-selected` + `[ci/run-xxx ...]` : Only run selected steps indicated by `ci/run-xxx` in your **DRAFT PR.** +- `ci/main-cron/run-all`: Run full `main-cron` workflow for your PR. +- `ci/main-cron/run-selected` + `[ci/run-xxx …]` : Run specific steps indicated by `ci/run-xxx` from the `main-cron` workflow, in your PR. Can use to verify some `main-cron` fix works as expected. - To reference `[ci/run-xxx ...]` labels, you may look at steps from `pull-request.yml` and `main-cron.yml`. -- **Be sure to add all the dependencies.** - - For example to run `e2e-test` for `main-cron` in your pull request: - 1. Add `ci/run-build`, `ci/run-build-other`, `ci/run-docslt` . - These correspond to its `depends` field in `pull-request.yml` and `main-cron.yml` . - 2. Add `ci/run-e2e-test` to run the step as well. - 3. Add `ci/run-main-cron` to run `main-cron` workflow in your pull request, - 4. Add `ci/main-cron/skip-ci` to skip all other steps which were not selected with `ci/run-xxx`. + +### Example + +https://github.com/risingwavelabs/risingwave/pull/17197 + +To run `e2e-test` and `e2e-source-test` for `main-cron` in your pull request: +1. Add `ci/run-e2e-test`. +2. Add `ci/run-e2e-source-tests`. +3. Add `ci/main-cron/run-selected` to skip all other steps which were not selected with `ci/run-xxx`. \ No newline at end of file From 1a92e5132cee859f3d25019ac45840c5c104960b Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 12 Jun 2024 15:36:05 +0800 Subject: [PATCH 20/37] refactor(jni): remove jni_core's dependency on storage (#17193) --- Cargo.lock | 13 +++- .../com/risingwave/java/binding/Binding.java | 6 +- .../java/binding/HummockIterator.java | 10 ++- src/java_binding/Cargo.toml | 19 +++++ .../src/hummock_iterator.rs | 43 +++++------ src/java_binding/src/lib.rs | 75 ++++++++++++++++++- src/jni_core/Cargo.toml | 3 - src/jni_core/src/jvm_runtime.rs | 33 ++++---- src/jni_core/src/lib.rs | 57 ++++++-------- src/jni_core/src/macros.rs | 22 ++++-- 10 files changed, 190 insertions(+), 91 deletions(-) rename src/{jni_core => java_binding}/src/hummock_iterator.rs (87%) diff --git a/Cargo.lock b/Cargo.lock index eb2ae2ed948cd..a3e602a653713 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11066,14 +11066,25 @@ dependencies = [ name = "risingwave_java_binding" version = "0.1.0" dependencies = [ + "anyhow", + "bytes", + "cfg-or-panic", + "foyer", + "futures", "jni", + "madsim-tokio", "prost 0.12.1", "risingwave_common", "risingwave_expr", + "risingwave_hummock_sdk", "risingwave_jni_core", + "risingwave_object_store", "risingwave_pb", + "risingwave_storage", + "rw_futures_util", "serde", "serde_json", + "tracing", ] [[package]] @@ -11096,9 +11107,7 @@ dependencies = [ "risingwave_common", "risingwave_expr", "risingwave_hummock_sdk", - "risingwave_object_store", "risingwave_pb", - "risingwave_storage", "rw_futures_util", "serde", "serde_json", diff --git a/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java b/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java index a16acda73e7fd..db832566fdfa7 100644 --- a/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java +++ b/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java @@ -26,6 +26,8 @@ public class Binding { } } + static void ensureInitialized() {} + public static native void tracingSlf4jEvent( String threadName, String name, int level, String message); @@ -33,10 +35,6 @@ public static native void tracingSlf4jEvent( public static native int vnodeCount(); - // hummock iterator method - // Return a pointer to the iterator - static native long iteratorNewHummock(byte[] readPlan); - static native long iteratorNewStreamChunk(long pointer); static native boolean iteratorNext(long pointer); diff --git a/java/java-binding/src/main/java/com/risingwave/java/binding/HummockIterator.java b/java/java-binding/src/main/java/com/risingwave/java/binding/HummockIterator.java index 03282a2dce528..a30391edbd380 100644 --- a/java/java-binding/src/main/java/com/risingwave/java/binding/HummockIterator.java +++ b/java/java-binding/src/main/java/com/risingwave/java/binding/HummockIterator.java @@ -20,8 +20,16 @@ public class HummockIterator implements AutoCloseable { private final long pointer; private boolean isClosed; + static { + Binding.ensureInitialized(); + } + + // hummock iterator method + // Return a pointer to the iterator + private static native long iteratorNewHummock(byte[] readPlan); + public HummockIterator(ReadPlan readPlan) { - this.pointer = Binding.iteratorNewHummock(readPlan.toByteArray()); + this.pointer = iteratorNewHummock(readPlan.toByteArray()); this.isClosed = false; } diff --git a/src/java_binding/Cargo.toml b/src/java_binding/Cargo.toml index 477f19878cbd9..0966b700a713f 100644 --- a/src/java_binding/Cargo.toml +++ b/src/java_binding/Cargo.toml @@ -10,13 +10,32 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] +anyhow = "1" +bytes = "1" +cfg-or-panic = "0.2" +foyer ={ workspace = true } +futures = { version = "0.3", default-features = false, features = ["alloc"] } jni = "0.21.1" prost = { workspace = true } risingwave_common = { workspace = true } +risingwave_hummock_sdk = { workspace = true } risingwave_jni_core = { workspace = true } +risingwave_object_store = { workspace = true } risingwave_pb = { workspace = true } +risingwave_storage = { workspace = true } +rw_futures_util = { workspace = true } serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" +tokio = { version = "0.2", package = "madsim-tokio", features = [ + "fs", + "rt", + "rt-multi-thread", + "sync", + "macros", + "time", + "signal", +] } +tracing = "0.1" [dev-dependencies] risingwave_expr = { workspace = true } diff --git a/src/jni_core/src/hummock_iterator.rs b/src/java_binding/src/hummock_iterator.rs similarity index 87% rename from src/jni_core/src/hummock_iterator.rs rename to src/java_binding/src/hummock_iterator.rs index c4445dece1314..4b6fc5b01742d 100644 --- a/src/jni_core/src/hummock_iterator.rs +++ b/src/java_binding/src/hummock_iterator.rs @@ -14,9 +14,10 @@ use std::sync::Arc; +use anyhow::anyhow; use bytes::Bytes; use foyer::HybridCacheBuilder; -use futures::{Stream, TryFutureExt, TryStreamExt}; +use futures::{TryFutureExt, TryStreamExt}; use risingwave_common::catalog::ColumnDesc; use risingwave_common::config::{MetricLevel, ObjectStoreConfig}; use risingwave_common::hash::VirtualNode; @@ -25,6 +26,7 @@ use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAw use risingwave_common::util::value_encoding::{BasicSerde, EitherSerde, ValueRowDeserializer}; use risingwave_hummock_sdk::key::{prefixed_range_with_vnode, TableKeyRange}; use risingwave_hummock_sdk::version::HummockVersion; +use risingwave_jni_core::HummockJavaBindingIterator; use risingwave_object_store::object::build_remote_object_store; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_pb::java_binding::key_range::Bound; @@ -39,35 +41,34 @@ use risingwave_storage::hummock::{ use risingwave_storage::monitor::{global_hummock_state_store_metrics, HummockStateStoreMetrics}; use risingwave_storage::row_serde::value_serde::ValueRowSerdeNew; use risingwave_storage::store::{ReadOptions, StateStoreIterExt}; -use risingwave_storage::table::KeyedRow; use rw_futures_util::select_all; use tokio::sync::mpsc::unbounded_channel; -type SelectAllIterStream = impl Stream>> + Unpin; -type SingleIterStream = impl Stream>>; +type SingleIterStream = HummockJavaBindingIterator; -fn select_all_vnode_stream(streams: Vec) -> SelectAllIterStream { - select_all(streams.into_iter().map(Box::pin)) +fn select_all_vnode_stream(streams: Vec) -> HummockJavaBindingIterator { + Box::pin(select_all(streams)) } fn to_deserialized_stream( iter: HummockStorageIterator, row_serde: EitherSerde, ) -> SingleIterStream { - iter.into_stream(move |(key, value)| { - Ok(KeyedRow::new( - key.user_key.table_key.copy_into(), - row_serde.deserialize(value).map(OwnedRow::new)?, - )) - }) + Box::pin( + iter.into_stream(move |(key, value)| { + Ok(( + Bytes::copy_from_slice(key.user_key.table_key.0), + row_serde.deserialize(value).map(OwnedRow::new)?, + )) + }) + .map_err(|e| anyhow!(e)), + ) } -pub struct HummockJavaBindingIterator { - stream: SelectAllIterStream, -} - -impl HummockJavaBindingIterator { - pub async fn new(read_plan: ReadPlan) -> StorageResult { +pub(crate) async fn new_hummock_java_binding_iter( + read_plan: ReadPlan, +) -> StorageResult { + { // Note(bugen): should we forward the implementation to the `StorageTable`? let object_store = Arc::new( build_remote_object_store( @@ -170,11 +171,7 @@ impl HummockJavaBindingIterator { let stream = select_all_vnode_stream(streams); - Ok(Self { stream }) - } - - pub async fn next(&mut self) -> StorageResult>> { - self.stream.try_next().await + Ok(stream) } } diff --git a/src/java_binding/src/lib.rs b/src/java_binding/src/lib.rs index ef5bb228b0cab..4fd089918bd5b 100644 --- a/src/java_binding/src/lib.rs +++ b/src/java_binding/src/lib.rs @@ -12,16 +12,83 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![feature(type_alias_impl_trait)] +#![feature(try_blocks)] + +mod hummock_iterator; use std::ffi::c_void; +use std::ops::Deref; +use anyhow::anyhow; +use cfg_or_panic::cfg_or_panic; +use jni::objects::JByteArray; use jni::sys::{jint, JNI_VERSION_1_2}; -use jni::JavaVM; -use risingwave_jni_core::register_native_method_for_jvm; +use jni::{JNIEnv, JavaVM}; +use prost::Message; +use risingwave_common::error::AsReport; +use risingwave_jni_core::jvm_runtime::{jvm_env, register_java_binding_native_methods}; +use risingwave_jni_core::{ + execute_and_catch, gen_class_name, to_guarded_slice, EnvParam, JavaBindingIterator, Pointer, + JAVA_BINDING_ASYNC_RUNTIME, +}; + +use crate::hummock_iterator::new_hummock_java_binding_iter; + +fn register_hummock_java_binding_native_methods( + env: &mut JNIEnv<'_>, +) -> Result<(), jni::errors::Error> { + let binding_class = env + .find_class(gen_class_name!(com.risingwave.java.binding.HummockIterator)) + .inspect_err(|e| tracing::error!(error = ?e.as_report(), "jvm find class error"))?; + macro_rules! gen_native_method_array { + () => {{ + risingwave_jni_core::split_extract_plain_native_methods! {{long iteratorNewHummock(byte[] readPlan);}, gen_native_method_array} + }}; + ({$({ $func_name:ident, {$($ret:tt)+}, {$($args:tt)*} })*}) => { + [ + $( + risingwave_jni_core::gen_native_method_entry! { + Java_com_risingwave_java_binding_HummockIterator_, $func_name, {$($ret)+}, {$($args)*} + }, + )* + ] + } + } + env.register_native_methods(binding_class, &gen_native_method_array!()) + .inspect_err( + |e| tracing::error!(error = ?e.as_report(), "jvm register native methods error"), + )?; + + tracing::info!("register native methods for jvm successfully"); + Ok(()) +} #[no_mangle] #[allow(non_snake_case)] pub extern "system" fn JNI_OnLoad(jvm: JavaVM, _reserved: *mut c_void) -> jint { - let _ = register_native_method_for_jvm(&jvm) - .inspect_err(|_e| eprintln!("unable to register native method")); + let result: Result<(), jni::errors::Error> = try { + let mut env = jvm_env(&jvm)?; + register_java_binding_native_methods(&mut env)?; + register_hummock_java_binding_native_methods(&mut env)?; + }; + let _ = + result.inspect_err(|e| eprintln!("unable to register native method: {:?}", e.as_report())); + JNI_VERSION_1_2 } + +#[cfg_or_panic(not(madsim))] +#[no_mangle] +extern "system" fn Java_com_risingwave_java_binding_HummockIterator_iteratorNewHummock<'a>( + env: EnvParam<'a>, + read_plan: JByteArray<'a>, +) -> Pointer<'static, JavaBindingIterator<'static>> { + execute_and_catch(env, move |env| { + let read_plan = Message::decode(to_guarded_slice(&read_plan, env)?.deref())?; + let iter = JAVA_BINDING_ASYNC_RUNTIME + .block_on(new_hummock_java_binding_iter(read_plan)) + .map_err(|e| anyhow!(e))?; + let iter = JavaBindingIterator::new_hummock_iter(iter); + Ok(iter.into()) + }) +} diff --git a/src/jni_core/Cargo.toml b/src/jni_core/Cargo.toml index 4d9c6cab092ab..a16776add6c6f 100644 --- a/src/jni_core/Cargo.toml +++ b/src/jni_core/Cargo.toml @@ -22,10 +22,7 @@ jni = { version = "0.21.1", features = ["invocation"] } paste = "1" prost = { workspace = true } risingwave_common = { workspace = true } -risingwave_hummock_sdk = { workspace = true } -risingwave_object_store = { workspace = true } risingwave_pb = { workspace = true } -risingwave_storage = { workspace = true } rw_futures_util = { workspace = true } serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" diff --git a/src/jni_core/src/jvm_runtime.rs b/src/jni_core/src/jvm_runtime.rs index f848b7d44240d..e596d5664dacb 100644 --- a/src/jni_core/src/jvm_runtime.rs +++ b/src/jni_core/src/jvm_runtime.rs @@ -20,8 +20,7 @@ use anyhow::{bail, Context}; use fs_err as fs; use fs_err::PathExt; use jni::objects::{JObject, JString}; -use jni::strings::JNIString; -use jni::{InitArgsBuilder, JNIEnv, JNIVersion, JavaVM, NativeMethod}; +use jni::{AttachGuard, InitArgsBuilder, JNIEnv, JNIVersion, JavaVM}; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use thiserror_ext::AsReport; use tracing::error; @@ -122,19 +121,27 @@ impl JavaVmWrapper { tracing::info!("initialize JVM successfully"); - register_native_method_for_jvm(&jvm).context("failed to register native method")?; + let result: std::result::Result<(), jni::errors::Error> = try { + let mut env = jvm_env(&jvm)?; + register_java_binding_native_methods(&mut env)?; + }; + + result.context("failed to register native method")?; Ok(jvm) } } -pub fn register_native_method_for_jvm(jvm: &JavaVM) -> Result<(), jni::errors::Error> { - let mut env = jvm - .attach_current_thread() - .inspect_err(|e| tracing::error!(error = ?e.as_report(), "jvm attach thread error"))?; +pub fn jvm_env(jvm: &JavaVM) -> Result, jni::errors::Error> { + jvm.attach_current_thread() + .inspect_err(|e| tracing::error!(error = ?e.as_report(), "jvm attach thread error")) +} +pub fn register_java_binding_native_methods( + env: &mut JNIEnv<'_>, +) -> Result<(), jni::errors::Error> { let binding_class = env - .find_class("com/risingwave/java/binding/Binding") + .find_class(gen_class_name!(com.risingwave.java.binding.Binding)) .inspect_err(|e| tracing::error!(error = ?e.as_report(), "jvm find class error"))?; use crate::*; macro_rules! gen_native_method_array { @@ -144,14 +151,8 @@ pub fn register_native_method_for_jvm(jvm: &JavaVM) -> Result<(), jni::errors::E ({$({ $func_name:ident, {$($ret:tt)+}, {$($args:tt)*} })*}) => { [ $( - { - let fn_ptr = paste::paste! {[ ]} as *mut c_void; - let sig = $crate::gen_jni_sig! { {$($ret)+}, {$($args)*}}; - NativeMethod { - name: JNIString::from(stringify! {$func_name}), - sig: JNIString::from(sig), - fn_ptr, - } + $crate::gen_native_method_entry! { + Java_com_risingwave_java_binding_Binding_, $func_name, {$($ret)+}, {$($args)*} }, )* ] diff --git a/src/jni_core/src/lib.rs b/src/jni_core/src/lib.rs index 7ff8e5aa930e8..18d1807948d21 100644 --- a/src/jni_core/src/lib.rs +++ b/src/jni_core/src/lib.rs @@ -18,7 +18,6 @@ #![feature(type_alias_impl_trait)] #![feature(try_blocks)] -pub mod hummock_iterator; pub mod jvm_runtime; mod macros; mod tracing_slf4j; @@ -33,6 +32,8 @@ use anyhow::anyhow; use bytes::Bytes; use cfg_or_panic::cfg_or_panic; use chrono::{Datelike, NaiveDateTime, Timelike}; +use futures::stream::BoxStream; +use futures::TryStreamExt; use jni::objects::{ AutoElements, GlobalRef, JByteArray, JClass, JMethodID, JObject, JStaticMethodID, JString, JValueOwned, ReleaseMode, @@ -42,6 +43,7 @@ use jni::sys::{ jboolean, jbyte, jdouble, jfloat, jint, jlong, jshort, jsize, jvalue, JNI_FALSE, JNI_TRUE, }; use jni::JNIEnv; +pub use paste::paste; use prost::{DecodeError, Message}; use risingwave_common::array::{ArrayError, StreamChunk}; use risingwave_common::hash::VirtualNode; @@ -54,17 +56,14 @@ use risingwave_pb::connector_service::{ SinkWriterStreamRequest, SinkWriterStreamResponse, }; use risingwave_pb::data::Op; -use risingwave_storage::error::StorageError; use thiserror::Error; use thiserror_ext::AsReport; use tokio::runtime::Runtime; use tokio::sync::mpsc::{Receiver, Sender}; use tracing_slf4j::*; -use crate::hummock_iterator::HummockJavaBindingIterator; -pub use crate::jvm_runtime::register_native_method_for_jvm; - -static RUNTIME: LazyLock = LazyLock::new(|| tokio::runtime::Runtime::new().unwrap()); +pub static JAVA_BINDING_ASYNC_RUNTIME: LazyLock = + LazyLock::new(|| tokio::runtime::Runtime::new().unwrap()); #[derive(Error, Debug)] pub enum BindingError { @@ -78,7 +77,7 @@ pub enum BindingError { #[error("StorageError {error}")] Storage { #[from] - error: StorageError, + error: anyhow::Error, backtrace: Backtrace, }, @@ -201,7 +200,7 @@ impl<'a> EnvParam<'a> { } } -fn execute_and_catch<'env, F, Ret>(mut env: EnvParam<'env>, inner: F) -> Ret +pub fn execute_and_catch<'env, F, Ret>(mut env: EnvParam<'env>, inner: F) -> Ret where F: FnOnce(&mut EnvParam<'env>) -> Result, Ret: Default + 'env, @@ -245,9 +244,10 @@ struct JavaClassMethodCache { } // TODO: may only return a RowRef -type StreamChunkRowIterator<'a> = impl Iterator + 'a; +pub type StreamChunkRowIterator<'a> = impl Iterator + 'a; +pub type HummockJavaBindingIterator = BoxStream<'static, anyhow::Result<(Bytes, OwnedRow)>>; -enum JavaBindingIteratorInner<'a> { +pub enum JavaBindingIteratorInner<'a> { Hummock(HummockJavaBindingIterator), StreamChunk(StreamChunkRowIterator<'a>), } @@ -288,12 +288,22 @@ struct RowCursor { extra: RowExtra, } -struct JavaBindingIterator<'a> { +pub struct JavaBindingIterator<'a> { inner: JavaBindingIteratorInner<'a>, cursor: Option, class_cache: JavaClassMethodCache, } +impl JavaBindingIterator<'static> { + pub fn new_hummock_iter(iter: HummockJavaBindingIterator) -> Self { + Self { + inner: JavaBindingIteratorInner::Hummock(iter), + cursor: None, + class_cache: Default::default(), + } + } +} + impl<'a> Deref for JavaBindingIterator<'a> { type Target = OwnedRow; @@ -311,24 +321,6 @@ extern "system" fn Java_com_risingwave_java_binding_Binding_vnodeCount(_env: Env VirtualNode::COUNT as jint } -#[cfg_or_panic(not(madsim))] -#[no_mangle] -extern "system" fn Java_com_risingwave_java_binding_Binding_iteratorNewHummock<'a>( - env: EnvParam<'a>, - read_plan: JByteArray<'a>, -) -> Pointer<'static, JavaBindingIterator<'static>> { - execute_and_catch(env, move |env| { - let read_plan = Message::decode(to_guarded_slice(&read_plan, env)?.deref())?; - let iter = RUNTIME.block_on(HummockJavaBindingIterator::new(read_plan))?; - let iter = JavaBindingIterator { - inner: JavaBindingIteratorInner::Hummock(iter), - cursor: None, - class_cache: Default::default(), - }; - Ok(iter.into()) - }) -} - #[cfg_or_panic(not(madsim))] #[no_mangle] extern "system" fn Java_com_risingwave_java_binding_Binding_iteratorNewStreamChunk<'a>( @@ -355,16 +347,15 @@ extern "system" fn Java_com_risingwave_java_binding_Binding_iteratorNext<'a>( let iter = pointer.as_mut(); match &mut iter.inner { JavaBindingIteratorInner::Hummock(ref mut hummock_iter) => { - match RUNTIME.block_on(hummock_iter.next())? { + match JAVA_BINDING_ASYNC_RUNTIME.block_on(hummock_iter.try_next())? { None => { iter.cursor = None; Ok(JNI_FALSE) } - Some(keyed_row) => { - let (key, row) = keyed_row.into_parts(); + Some((key, row)) => { iter.cursor = Some(RowCursor { row, - extra: RowExtra::Key(key.0), + extra: RowExtra::Key(key), }); Ok(JNI_TRUE) } diff --git a/src/jni_core/src/macros.rs b/src/jni_core/src/macros.rs index 1b2f79f829564..982ccda06ecf0 100644 --- a/src/jni_core/src/macros.rs +++ b/src/jni_core/src/macros.rs @@ -448,10 +448,6 @@ macro_rules! for_all_plain_native_methods { public static native int vnodeCount(); - // hummock iterator method - // Return a pointer to the iterator - static native long iteratorNewHummock(byte[] readPlan); - static native long iteratorNewStreamChunk(long pointer); static native boolean iteratorNext(long pointer); @@ -839,6 +835,23 @@ macro_rules! call_method { }}; } +#[macro_export] +macro_rules! gen_native_method_entry { + ( + $class_prefix:ident, $func_name:ident, {$($ret:tt)+}, {$($args:tt)*} + ) => {{ + { + let fn_ptr = $crate::paste! {[<$class_prefix $func_name> ]} as *mut c_void; + let sig = $crate::gen_jni_sig! { {$($ret)+}, {$($args)*}}; + jni::NativeMethod { + name: jni::strings::JNIString::from(stringify! {$func_name}), + sig: jni::strings::JNIString::from(sig), + fn_ptr, + } + } + }}; +} + #[cfg(test)] mod tests { use std::fmt::Formatter; @@ -891,7 +904,6 @@ mod tests { tracingSlf4jEvent (Ljava/lang/String;Ljava/lang/String;ILjava/lang/String;)V, tracingSlf4jEventEnabled (I)Z, vnodeCount ()I, - iteratorNewHummock ([B)J, iteratorNewStreamChunk (J)J, iteratorNext (J)Z, iteratorClose (J)V, From 6bd2360e0ee9750f40bf93fa9cc02c3ece75af7a Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 12 Jun 2024 15:36:48 +0800 Subject: [PATCH 21/37] fix(batch): fix memory limit for batch sort executor (#17211) --- src/batch/src/executor/merge_sort_exchange.rs | 16 ++++------------ src/batch/src/executor/order_by.rs | 15 ++++++++++++++- 2 files changed, 18 insertions(+), 13 deletions(-) diff --git a/src/batch/src/executor/merge_sort_exchange.rs b/src/batch/src/executor/merge_sort_exchange.rs index 3c0f13198a3ae..e2779967dbcbe 100644 --- a/src/batch/src/executor/merge_sort_exchange.rs +++ b/src/batch/src/executor/merge_sort_exchange.rs @@ -123,7 +123,7 @@ impl MergeSortExchangeEx // Check whether there is indeed a chunk and there is a visible row sitting at `row_idx` // in the chunk before calling this function. - fn push_row_into_heap(&mut self, source_idx: usize, row_idx: usize) -> Result<()> { + fn push_row_into_heap(&mut self, source_idx: usize, row_idx: usize) { assert!(source_idx < self.source_inputs.len()); let chunk_ref = self.source_inputs[source_idx].as_ref().unwrap(); self.min_heap.push(HeapElem::new( @@ -133,14 +133,6 @@ impl MergeSortExchangeEx row_idx, None, )); - - if self.min_heap.mem_context().check_memory_usage() { - Ok(()) - } else { - Err(BatchError::OutOfMemory( - self.min_heap.mem_context().mem_limit(), - )) - } } } @@ -176,7 +168,7 @@ impl MergeSortExchangeEx // exchange, therefore we are sure that there is at least // one visible row. let next_row_idx = chunk.next_visible_row_idx(0); - self.push_row_into_heap(source_idx, next_row_idx.unwrap())?; + self.push_row_into_heap(source_idx, next_row_idx.unwrap()); } } @@ -211,13 +203,13 @@ impl MergeSortExchangeEx let possible_next_row_idx = cur_chunk.next_visible_row_idx(row_idx + 1); match possible_next_row_idx { Some(next_row_idx) => { - self.push_row_into_heap(child_idx, next_row_idx)?; + self.push_row_into_heap(child_idx, next_row_idx); } None => { self.get_source_chunk(child_idx).await?; if let Some(chunk) = &self.source_inputs[child_idx] { let next_row_idx = chunk.next_visible_row_idx(0); - self.push_row_into_heap(child_idx, next_row_idx.unwrap())?; + self.push_row_into_heap(child_idx, next_row_idx.unwrap()); } } } diff --git a/src/batch/src/executor/order_by.rs b/src/batch/src/executor/order_by.rs index fd07b4fab845e..05cd0f8c94fa0 100644 --- a/src/batch/src/executor/order_by.rs +++ b/src/batch/src/executor/order_by.rs @@ -19,6 +19,7 @@ use risingwave_common::memory::MemoryContext; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::memcmp_encoding::encode_chunk; use risingwave_common::util::sort_util::ColumnOrder; +use risingwave_common_estimate_size::EstimateSize; use risingwave_pb::batch_plan::plan_node::NodeBody; use super::{BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder}; @@ -91,7 +92,12 @@ impl SortExecutor { #[for_await] for chunk in self.child.execute() { - chunks.push(chunk?.compact()); + let chunk = chunk?.compact(); + let chunk_estimated_heap_size = chunk.estimated_heap_size(); + chunks.push(chunk); + if !self.mem_context.add(chunk_estimated_heap_size as i64) { + Err(BatchError::OutOfMemory(self.mem_context.mem_limit()))?; + } } let mut encoded_rows = @@ -99,12 +105,19 @@ impl SortExecutor { for chunk in &chunks { let encoded_chunk = encode_chunk(chunk, &self.column_orders)?; + let chunk_estimated_heap_size = encoded_chunk + .iter() + .map(|x| x.estimated_heap_size()) + .sum::(); encoded_rows.extend( encoded_chunk .into_iter() .enumerate() .map(|(row_id, row)| (chunk.row_at_unchecked_vis(row_id), row)), ); + if !self.mem_context.add(chunk_estimated_heap_size as i64) { + Err(BatchError::OutOfMemory(self.mem_context.mem_limit()))?; + } } encoded_rows.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); From 3da3d16c742b90f1c08e6180b9e19b46cf6d31e5 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 12 Jun 2024 15:36:58 +0800 Subject: [PATCH 22/37] refactor(storage): limit use of hummock version safe epoch (#17161) --- .../rw_catalog/rw_hummock_version.rs | 2 +- .../backup_restore/meta_snapshot_builder.rs | 6 +- src/meta/src/backup_restore/restore.rs | 14 +- src/meta/src/hummock/manager/commit_epoch.rs | 26 ++-- src/meta/src/hummock/manager/compaction.rs | 43 +++--- .../manager/compaction_group_manager.rs | 9 +- src/meta/src/hummock/manager/mod.rs | 2 +- src/meta/src/hummock/manager/transaction.rs | 7 +- src/meta/src/hummock/manager/versioning.rs | 44 +----- src/storage/backup/src/lib.rs | 2 +- src/storage/benches/bench_table_watermarks.rs | 38 +++-- .../compaction_group/hummock_version_ext.rs | 139 ++++++++---------- .../hummock_sdk/src/table_watermark.rs | 7 +- src/storage/hummock_sdk/src/version.rs | 68 ++++++++- src/storage/src/hummock/error.rs | 16 +- .../src/hummock/event_handler/uploader.rs | 9 +- .../hummock/local_version/pinned_version.rs | 4 - .../src/hummock/store/hummock_storage.rs | 12 +- src/storage/src/hummock/utils.rs | 16 +- 19 files changed, 260 insertions(+), 204 deletions(-) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs index e3c0578ac6864..f7d265485f706 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs @@ -106,7 +106,7 @@ fn version_to_compaction_group_rows(version: &HummockVersion) -> Vec HummockVersionTransaction<'a> { }; group_deltas.push(group_delta); version_delta.safe_epoch = std::cmp::max( - version_delta.latest_version().safe_epoch, + version_delta.latest_version().visible_table_safe_epoch(), compact_task.watermark, ); - if version_delta.latest_version().safe_epoch < version_delta.safe_epoch { - version_delta.state_table_info_delta = version_delta - .latest_version() - .state_table_info - .info() - .iter() - .map(|(table_id, info)| { - ( - *table_id, - StateTableInfoDelta { - committed_epoch: info.committed_epoch, - safe_epoch: version_delta.safe_epoch, - }, - ) - }) - .collect(); + if version_delta.latest_version().visible_table_safe_epoch() < version_delta.safe_epoch { + version_delta.with_latest_version(|version, version_delta| { + for (table_id, info) in version.state_table_info.info() { + let new_safe_epoch = min(version_delta.safe_epoch, info.committed_epoch); + if new_safe_epoch > info.safe_epoch { + if new_safe_epoch != version_delta.safe_epoch { + warn!( + new_safe_epoch, + committed_epoch = info.committed_epoch, + global_safe_epoch = version_delta.safe_epoch, + table_id = table_id.table_id, + "table has different safe epoch to global" + ); + } + version_delta.state_table_info_delta.insert( + *table_id, + StateTableInfoDelta { + committed_epoch: info.committed_epoch, + safe_epoch: new_safe_epoch, + }, + ); + } + } + }); } version_delta.pre_apply(); } diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index 7d56e2f9dc0d9..df5189d539180 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -188,10 +188,7 @@ impl HummockManager { &self.metrics, ); let mut new_version_delta = version.new_delta(); - let (committed_epoch, safe_epoch) = { - let version = new_version_delta.latest_version(); - (version.max_committed_epoch, version.safe_epoch) - }; + let epoch = new_version_delta.latest_version().max_committed_epoch; for (table_id, raw_group_id) in pairs { let mut group_id = *raw_group_id; @@ -247,8 +244,8 @@ impl HummockManager { .insert( TableId::new(*table_id), StateTableInfoDelta { - committed_epoch, - safe_epoch, + committed_epoch: epoch, + safe_epoch: epoch, } ) .is_none()); diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 1c0bbde4eb335..47209ddf1fff2 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -392,7 +392,7 @@ impl HummockManager { .read() .await .default_compaction_config(); - let checkpoint_version = create_init_version(default_compaction_config); + let checkpoint_version = HummockVersion::create_init_version(default_compaction_config); tracing::info!("init hummock version checkpoint"); versioning_guard.checkpoint = HummockVersionCheckpoint { version: checkpoint_version.clone(), diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index e5f2ba4b325be..c467e95adfdbe 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -15,7 +15,6 @@ use std::collections::BTreeMap; use std::ops::{Deref, DerefMut}; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::build_version_delta_after_version; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::HummockVersionId; use risingwave_pb::hummock::HummockVersionStats; @@ -38,7 +37,9 @@ fn trigger_version_stat(metrics: &MetaMetrics, current_version: &HummockVersion) metrics .version_size .set(current_version.estimated_encode_len() as i64); - metrics.safe_epoch.set(current_version.safe_epoch as i64); + metrics + .safe_epoch + .set(current_version.visible_table_safe_epoch() as i64); metrics.current_version_id.set(current_version.id as i64); } @@ -86,7 +87,7 @@ impl<'a> HummockVersionTransaction<'a> { } pub(super) fn new_delta<'b>(&'b mut self) -> SingleDeltaTransaction<'a, 'b> { - let delta = build_version_delta_after_version(self.latest_version()); + let delta = self.latest_version().version_delta_after(); SingleDeltaTransaction { version_txn: self, delta: Some(delta), diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index e04af41d30d40..790ac6b54fef1 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -17,25 +17,20 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use itertools::Itertools; use risingwave_common::catalog::TableId; -use risingwave_common::util::epoch::INVALID_EPOCH; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ - build_initial_compaction_group_levels, get_compaction_group_ids, - get_table_compaction_group_id_mapping, BranchedSstInfo, + get_compaction_group_ids, get_table_compaction_group_id_mapping, BranchedSstInfo, }; -use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; +use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::table_stats::add_prost_table_stats_map; -use risingwave_hummock_sdk::version::{ - HummockVersion, HummockVersionDelta, HummockVersionStateTableInfo, -}; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{ CompactionGroupId, HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, - FIRST_VERSION_ID, }; use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ - CompactionConfig, HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, - HummockVersionStats, SstableInfo, TableStats, + HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, HummockVersionStats, SstableInfo, + TableStats, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -349,28 +344,6 @@ pub(super) fn calc_new_write_limits( new_write_limits } -pub(super) fn create_init_version(default_compaction_config: CompactionConfig) -> HummockVersion { - let mut init_version = HummockVersion { - id: FIRST_VERSION_ID, - levels: Default::default(), - max_committed_epoch: INVALID_EPOCH, - safe_epoch: INVALID_EPOCH, - table_watermarks: HashMap::new(), - table_change_log: HashMap::new(), - state_table_info: HummockVersionStateTableInfo::empty(), - }; - for group_id in [ - StaticCompactionGroupId::StateDefault as CompactionGroupId, - StaticCompactionGroupId::MaterializedView as CompactionGroupId, - ] { - init_version.levels.insert( - group_id, - build_initial_compaction_group_levels(group_id, &default_compaction_config), - ); - } - init_version -} - /// Rebuilds table stats from the given version. /// Note that the result is approximate value. See `estimate_table_stats`. fn rebuild_table_stats(version: &HummockVersion) -> HummockVersionStats { @@ -575,10 +548,9 @@ mod tests { ); } - let mut version = HummockVersion { - id: 123, - ..Default::default() - }; + let mut version = HummockVersion::default(); + version.id = 123; + for cg in 1..3 { version.levels.insert( cg, diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index a1acfde20e400..56c3ed551a282 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -71,7 +71,7 @@ impl MetaSnapshotMetadata { hummock_version_id: v.id, ssts: v.get_object_ids(), max_committed_epoch: v.max_committed_epoch, - safe_epoch: v.safe_epoch, + safe_epoch: v.visible_table_safe_epoch(), format_version, remarks, } diff --git a/src/storage/benches/bench_table_watermarks.rs b/src/storage/benches/bench_table_watermarks.rs index 11ec3c4bdcb54..ce8980598f772 100644 --- a/src/storage/benches/bench_table_watermarks.rs +++ b/src/storage/benches/bench_table_watermarks.rs @@ -15,7 +15,7 @@ #![feature(lazy_cell)] use std::collections::hash_map::Entry; -use std::collections::{HashMap, VecDeque}; +use std::collections::{HashMap, HashSet, VecDeque}; use std::sync::{Arc, LazyLock}; use bytes::Bytes; @@ -28,8 +28,9 @@ use risingwave_common::util::epoch::test_epoch; use risingwave_hummock_sdk::table_watermark::{ TableWatermarks, TableWatermarksIndex, VnodeWatermark, WatermarkDirection, }; -use risingwave_hummock_sdk::version::HummockVersion; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionStateTableInfo}; use risingwave_hummock_sdk::HummockEpoch; +use risingwave_pb::hummock::StateTableInfoDelta; use risingwave_storage::hummock::local_version::pinned_version::PinnedVersion; use spin::Mutex; use tokio::sync::mpsc::unbounded_channel; @@ -115,17 +116,30 @@ fn gen_version( new_epoch_idx, vnode_part_count, )); - // let table_watermarks = - // gen_committed_table_watermarks(old_epoch_idx, new_epoch_idx, vnode_part_count); - HummockVersion { - id: new_epoch_idx as _, - max_committed_epoch: test_epoch(new_epoch_idx as _), - safe_epoch: test_epoch(old_epoch_idx as _), - table_watermarks: (0..table_count) - .map(|table_id| (TableId::new(table_id as _), table_watermarks.clone())) + let mut version = HummockVersion::default(); + let committed_epoch = test_epoch(new_epoch_idx as _); + version.id = new_epoch_idx as _; + version.max_committed_epoch = committed_epoch; + version.table_watermarks = (0..table_count) + .map(|table_id| (TableId::new(table_id as _), table_watermarks.clone())) + .collect(); + let mut state_table_info = HummockVersionStateTableInfo::empty(); + state_table_info.apply_delta( + &(0..table_count) + .map(|table_id| { + ( + TableId::new(table_id as _), + StateTableInfoDelta { + committed_epoch, + safe_epoch: test_epoch(old_epoch_idx as _), + }, + ) + }) .collect(), - ..Default::default() - } + &HashSet::new(), + ); + version.state_table_info = state_table_info; + version } fn bench_table_watermarks(c: &mut Criterion) { diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index 52f3c1cb15ca1..2231878dc9ef6 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -246,8 +246,15 @@ impl HummockVersion { if !existing_table_ids.contains(&u32_table_id) { None } else { - extract_single_table_watermark(table_watermarks, self.safe_epoch) - .map(|table_watermarks| (table_id.table_id, table_watermarks)) + extract_single_table_watermark( + table_watermarks, + self.state_table_info + .info() + .get(table_id) + .expect("table should exist") + .safe_epoch, + ) + .map(|table_watermarks| (table_id.table_id, table_watermarks)) } }) .collect() @@ -584,7 +591,7 @@ impl HummockVersion { } self.id = version_delta.id; self.max_committed_epoch = version_delta.max_committed_epoch; - self.safe_epoch = version_delta.safe_epoch; + self.set_safe_epoch(version_delta.safe_epoch); // apply to table watermark @@ -1116,21 +1123,6 @@ pub fn insert_new_sub_level( l0.sub_levels.insert(insert_pos, level); } -pub fn build_version_delta_after_version(version: &HummockVersion) -> HummockVersionDelta { - HummockVersionDelta { - id: version.next_version_id(), - prev_id: version.id, - safe_epoch: version.safe_epoch, - trivial_move: false, - max_committed_epoch: version.max_committed_epoch, - group_deltas: Default::default(), - new_table_watermarks: HashMap::new(), - removed_table_ids: HashSet::new(), - change_log_delta: HashMap::new(), - state_table_info_delta: Default::default(), - } -} - /// Delete sstables if the table id is in the id set. /// /// Return `true` if some sst is deleted, and `false` is the deletion is trivial @@ -1209,10 +1201,11 @@ pub fn validate_version(version: &HummockVersion) -> Vec { let mut res = Vec::new(); // Ensure safe_epoch <= max_committed_epoch - if version.safe_epoch > version.max_committed_epoch { + if version.visible_table_safe_epoch() > version.max_committed_epoch { res.push(format!( "VERSION: safe_epoch {} > max_committed_epoch {}", - version.safe_epoch, version.max_committed_epoch + version.visible_table_safe_epoch(), + version.max_committed_epoch )); } @@ -1353,22 +1346,20 @@ mod tests { #[test] fn test_get_sst_object_ids() { - let mut version = HummockVersion { - id: 0, - levels: HashMap::from_iter([( - 0, - Levels { - levels: vec![], - l0: Some(OverlappingLevel { - sub_levels: vec![], - total_file_size: 0, - uncompressed_file_size: 0, - }), - ..Default::default() - }, - )]), - ..Default::default() - }; + let mut version = HummockVersion::default(); + version.id = 0; + version.levels = HashMap::from_iter([( + 0, + Levels { + levels: vec![], + l0: Some(OverlappingLevel { + sub_levels: vec![], + total_file_size: 0, + uncompressed_file_size: 0, + }), + ..Default::default() + }, + )]); assert_eq!(version.get_object_ids().len(), 0); // Add to sub level @@ -1404,32 +1395,30 @@ mod tests { #[test] fn test_apply_version_delta() { - let mut version = HummockVersion { - id: 0, - levels: HashMap::from_iter([ - ( + let mut version = HummockVersion::default(); + version.id = 0; + version.levels = HashMap::from_iter([ + ( + 0, + build_initial_compaction_group_levels( 0, - build_initial_compaction_group_levels( - 0, - &CompactionConfig { - max_level: 6, - ..Default::default() - }, - ), + &CompactionConfig { + max_level: 6, + ..Default::default() + }, ), - ( + ), + ( + 1, + build_initial_compaction_group_levels( 1, - build_initial_compaction_group_levels( - 1, - &CompactionConfig { - max_level: 6, - ..Default::default() - }, - ), + &CompactionConfig { + max_level: 6, + ..Default::default() + }, ), - ]), - ..Default::default() - }; + ), + ]); let version_delta = HummockVersionDelta { id: 1, group_deltas: HashMap::from_iter([ @@ -1492,25 +1481,23 @@ mod tests { }], ..Default::default() }; - assert_eq!( - version, - HummockVersion { - id: 1, - levels: HashMap::from_iter([ - ( + assert_eq!(version, { + let mut version = HummockVersion::default(); + version.id = 1; + version.levels = HashMap::from_iter([ + ( + 2, + build_initial_compaction_group_levels( 2, - build_initial_compaction_group_levels( - 2, - &CompactionConfig { - max_level: 6, - ..Default::default() - } - ), + &CompactionConfig { + max_level: 6, + ..Default::default() + }, ), - (1, cg1,), - ]), - ..Default::default() - } - ); + ), + (1, cg1), + ]); + version + }); } } diff --git a/src/storage/hummock_sdk/src/table_watermark.rs b/src/storage/hummock_sdk/src/table_watermark.rs index 24d28dec9ba1c..73f7bac358e1b 100644 --- a/src/storage/hummock_sdk/src/table_watermark.rs +++ b/src/storage/hummock_sdk/src/table_watermark.rs @@ -1031,11 +1031,8 @@ mod tests { watermark3.clone(), ); - let mut version = HummockVersion { - max_committed_epoch: EPOCH1, - safe_epoch: EPOCH1, - ..Default::default() - }; + let mut version = HummockVersion::default(); + version.max_committed_epoch = EPOCH1; let test_table_id = TableId::from(233); version.table_watermarks.insert( test_table_id, diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 130c5f6f523da..51780b1bc7334 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -19,18 +19,22 @@ use std::sync::Arc; use prost::Message; use risingwave_common::catalog::TableId; +use risingwave_common::util::epoch::INVALID_EPOCH; use risingwave_pb::hummock::group_delta::DeltaType; use risingwave_pb::hummock::hummock_version::Levels as PbLevels; use risingwave_pb::hummock::hummock_version_delta::{ChangeLogDelta, GroupDeltas as PbGroupDeltas}; use risingwave_pb::hummock::{ - HummockVersion as PbHummockVersion, HummockVersionDelta as PbHummockVersionDelta, SstableInfo, - StateTableInfo as PbStateTableInfo, StateTableInfo, StateTableInfoDelta, + CompactionConfig, HummockVersion as PbHummockVersion, + HummockVersionDelta as PbHummockVersionDelta, SstableInfo, StateTableInfo as PbStateTableInfo, + StateTableInfo, StateTableInfoDelta, }; use tracing::warn; use crate::change_log::TableChangeLog; +use crate::compaction_group::hummock_version_ext::build_initial_compaction_group_levels; +use crate::compaction_group::StaticCompactionGroupId; use crate::table_watermark::TableWatermarks; -use crate::{CompactionGroupId, HummockSstableObjectId, HummockVersionId}; +use crate::{CompactionGroupId, HummockSstableObjectId, HummockVersionId, FIRST_VERSION_ID}; #[derive(Debug, Clone, PartialEq)] pub struct HummockVersionStateTableInfo { @@ -85,7 +89,16 @@ impl HummockVersionStateTableInfo { }; match self.state_table_info.entry(*table_id) { Entry::Occupied(mut entry) => { - let prev_info = replace(entry.get_mut(), new_info); + let prev_info = entry.get_mut(); + assert!( + new_info.safe_epoch >= prev_info.safe_epoch + && new_info.committed_epoch >= prev_info.committed_epoch, + "state table info regress. table id: {}, prev_info: {:?}, new_info: {:?}", + table_id.table_id, + prev_info, + new_info + ); + let prev_info = replace(prev_info, new_info); changed_table.insert(*table_id, Some(prev_info)); } Entry::Vacant(entry) => { @@ -107,7 +120,7 @@ pub struct HummockVersion { pub id: u64, pub levels: HashMap, pub max_committed_epoch: u64, - pub safe_epoch: u64, + safe_epoch: u64, pub table_watermarks: HashMap>, pub table_change_log: HashMap, pub state_table_info: HummockVersionStateTableInfo, @@ -244,6 +257,51 @@ impl HummockVersion { } } } + + pub(crate) fn set_safe_epoch(&mut self, safe_epoch: u64) { + self.safe_epoch = safe_epoch; + } + + pub fn visible_table_safe_epoch(&self) -> u64 { + self.safe_epoch + } + + pub fn create_init_version(default_compaction_config: CompactionConfig) -> HummockVersion { + let mut init_version = HummockVersion { + id: FIRST_VERSION_ID, + levels: Default::default(), + max_committed_epoch: INVALID_EPOCH, + safe_epoch: INVALID_EPOCH, + table_watermarks: HashMap::new(), + table_change_log: HashMap::new(), + state_table_info: HummockVersionStateTableInfo::empty(), + }; + for group_id in [ + StaticCompactionGroupId::StateDefault as CompactionGroupId, + StaticCompactionGroupId::MaterializedView as CompactionGroupId, + ] { + init_version.levels.insert( + group_id, + build_initial_compaction_group_levels(group_id, &default_compaction_config), + ); + } + init_version + } + + pub fn version_delta_after(&self) -> HummockVersionDelta { + HummockVersionDelta { + id: self.next_version_id(), + prev_id: self.id, + safe_epoch: self.safe_epoch, + trivial_move: false, + max_committed_epoch: self.max_committed_epoch, + group_deltas: Default::default(), + new_table_watermarks: HashMap::new(), + removed_table_ids: HashSet::new(), + change_log_delta: HashMap::new(), + state_table_info_delta: Default::default(), + } + } } #[derive(Debug, PartialEq, Clone)] diff --git a/src/storage/src/hummock/error.rs b/src/storage/src/hummock/error.rs index 3019e65fc4e36..48f71b9199332 100644 --- a/src/storage/src/hummock/error.rs +++ b/src/storage/src/hummock/error.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::catalog::TableId; use risingwave_object_store::object::ObjectError; use thiserror::Error; use thiserror_ext::AsReport; @@ -48,7 +49,11 @@ pub enum HummockErrorInner { #[error("Barrier read is unavailable for now. Likely the cluster is recovering")] ReadCurrentEpoch, #[error("Expired Epoch: watermark {safe_epoch}, epoch {epoch}")] - ExpiredEpoch { safe_epoch: u64, epoch: u64 }, + ExpiredEpoch { + table_id: u32, + safe_epoch: u64, + epoch: u64, + }, #[error("CompactionExecutor error: {0}")] CompactionExecutor(String), #[error("FileCache error: {0}")] @@ -108,8 +113,13 @@ impl HummockError { HummockErrorInner::ReadCurrentEpoch.into() } - pub fn expired_epoch(safe_epoch: u64, epoch: u64) -> HummockError { - HummockErrorInner::ExpiredEpoch { safe_epoch, epoch }.into() + pub fn expired_epoch(table_id: TableId, safe_epoch: u64, epoch: u64) -> HummockError { + HummockErrorInner::ExpiredEpoch { + table_id: table_id.table_id, + safe_epoch, + epoch, + } + .into() } pub fn is_expired_epoch(&self) -> bool { diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index 9f40e9ac28962..1feaf017b2f02 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -1176,11 +1176,10 @@ pub(crate) mod tests { } fn test_hummock_version(epoch: HummockEpoch) -> HummockVersion { - HummockVersion { - id: epoch, - max_committed_epoch: epoch, - ..Default::default() - } + let mut version = HummockVersion::default(); + version.id = epoch; + version.max_committed_epoch = epoch; + version } fn initial_pinned_version() -> PinnedVersion { diff --git a/src/storage/src/hummock/local_version/pinned_version.rs b/src/storage/src/hummock/local_version/pinned_version.rs index da9569e6bb83c..6302f91739c20 100644 --- a/src/storage/src/hummock/local_version/pinned_version.rs +++ b/src/storage/src/hummock/local_version/pinned_version.rs @@ -155,10 +155,6 @@ impl PinnedVersion { self.version.max_committed_epoch } - pub fn safe_epoch(&self) -> u64 { - self.version.safe_epoch - } - /// ret value can't be used as `HummockVersion`. it must be modified with delta pub fn version(&self) -> &HummockVersion { &self.version diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index bfe24e5559376..e52b87c7d8aba 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -29,7 +29,7 @@ use risingwave_hummock_sdk::key::{ is_empty_key_range, vnode, vnode_range, TableKey, TableKeyRange, }; use risingwave_hummock_sdk::table_watermark::TableWatermarksIndex; -use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; +use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_pb::hummock::SstableInfo; use risingwave_rpc_client::HummockMetaClient; use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; @@ -313,7 +313,8 @@ impl HummockStorage { ) -> StorageResult<(TableKeyRange, ReadVersionTuple)> { match self.backup_reader.try_get_hummock_version(epoch).await { Ok(Some(backup_version)) => { - validate_safe_epoch(backup_version.safe_epoch(), epoch)?; + validate_safe_epoch(backup_version.version(), table_id, epoch)?; + Ok(get_committed_read_version_tuple( backup_version, table_id, @@ -337,7 +338,7 @@ impl HummockStorage { key_range: TableKeyRange, ) -> StorageResult<(TableKeyRange, ReadVersionTuple)> { let pinned_version = self.pinned_version.load(); - validate_safe_epoch(pinned_version.safe_epoch(), epoch)?; + validate_safe_epoch(pinned_version.version(), table_id, epoch)?; // check epoch if lower mce let ret = if epoch <= pinned_version.max_committed_epoch() { @@ -643,7 +644,10 @@ use risingwave_hummock_sdk::version::HummockVersion; #[cfg(any(test, feature = "test"))] impl HummockStorage { - pub async fn seal_and_sync_epoch(&self, epoch: u64) -> StorageResult { + pub async fn seal_and_sync_epoch( + &self, + epoch: u64, + ) -> StorageResult { self.seal_epoch(epoch, true); self.sync(epoch).await } diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 4d61e7cd33674..4c270ee736b97 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -70,9 +70,19 @@ where !too_left && !too_right } -pub fn validate_safe_epoch(safe_epoch: u64, epoch: u64) -> HummockResult<()> { - if epoch < safe_epoch { - return Err(HummockError::expired_epoch(safe_epoch, epoch)); +pub fn validate_safe_epoch( + version: &HummockVersion, + table_id: TableId, + epoch: u64, +) -> HummockResult<()> { + if let Some(info) = version.state_table_info.info().get(&table_id) + && epoch < info.safe_epoch + { + return Err(HummockError::expired_epoch( + table_id, + info.safe_epoch, + epoch, + )); } Ok(()) From 6ab7752a9639049fc75de9e9f3a95b342c3d69af Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 12 Jun 2024 15:48:35 +0800 Subject: [PATCH 23/37] chore(ci): disable e2e visibility mode tests in single node mode (#17215) --- ci/scripts/run-e2e-test.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index 94398a77a8300..4736f4aa53a8a 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -92,7 +92,11 @@ sqllogictest -p 4566 -d dev './e2e_test/ddl/**/*.slt' --junit "batch-ddl-${profi if [[ "$mode" != "single-node" ]]; then sqllogictest -p 4566 -d dev './e2e_test/background_ddl/basic.slt' --junit "batch-ddl-${profile}" fi -sqllogictest -p 4566 -d dev './e2e_test/visibility_mode/*.slt' --junit "batch-${profile}" + +if [[ $mode != "single-node" ]]; then + sqllogictest -p 4566 -d dev './e2e_test/visibility_mode/*.slt' --junit "batch-${profile}" +fi + sqllogictest -p 4566 -d dev './e2e_test/ttl/ttl.slt' sqllogictest -p 4566 -d dev './e2e_test/database/prepare.slt' sqllogictest -p 4566 -d test './e2e_test/database/test.slt' From c491b46e74455de22ed5c742d5e14d4c12e2a83b Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Wed, 12 Jun 2024 16:48:23 +0800 Subject: [PATCH 24/37] chore: close inactive PRs and allow manual trigger (#17220) --- .github/workflows/stale.yml | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index eb5173e599e7d..cdd8b591d65c8 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -3,17 +3,23 @@ name: Mark stale issues and pull requests on: schedule: - cron: '30 1 * * *' + workflow_dispatch: + inputs: + # https://github.com/marketplace/actions/close-stale-issues#operations-per-run + operationsPerRun: + description: 'Max number of operations per run' + required: true + default: 30 jobs: stale: - runs-on: ubuntu-latest permissions: issues: write pull-requests: write steps: - - uses: actions/stale@v5 + - uses: actions/stale@v9 with: repo-token: ${{ secrets.GITHUB_TOKEN }} stale-issue-message: > @@ -25,3 +31,6 @@ jobs: stale-issue-label: 'no-issue-activity' stale-pr-label: 'no-pr-activity' days-before-close: -1 + days-before-pr-close: 7 + operations-per-run: ${{ github.event.inputs.operationsPerRun }} + enable-statistics: true From c9c706415b52083205a91b228ae53a4bc662e97d Mon Sep 17 00:00:00 2001 From: lmatz Date: Wed, 12 Jun 2024 18:09:17 +0800 Subject: [PATCH 25/37] fix(metrics): reduce overhead of `merge_barrier_alignment_duration` (#17222) --- src/stream/src/executor/merge.rs | 48 ++++++++++++++++++++------------ 1 file changed, 30 insertions(+), 18 deletions(-) diff --git a/src/stream/src/executor/merge.rs b/src/stream/src/executor/merge.rs index 8b1daa9695c4d..19124fe8c22d4 100644 --- a/src/stream/src/executor/merge.rs +++ b/src/stream/src/executor/merge.rs @@ -18,6 +18,9 @@ use std::task::{Context, Poll}; use anyhow::Context as _; use futures::stream::{FusedStream, FuturesUnordered, StreamFuture}; +use prometheus::Histogram; +use risingwave_common::config::MetricLevel; +use risingwave_common::metrics::LabelGuardedMetric; use tokio::time::Instant; use super::exchange::input::BoxedInput; @@ -92,12 +95,24 @@ impl MergeExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self: Box) { + let merge_barrier_align_duration = if self.metrics.level >= MetricLevel::Debug { + Some( + self.metrics + .merge_barrier_align_duration + .with_label_values(&[ + &self.actor_context.id.to_string(), + &self.actor_context.fragment_id.to_string(), + ]), + ) + } else { + None + }; + // Futures of all active upstreams. let select_all = SelectReceivers::new( self.actor_context.id, - self.actor_context.fragment_id, self.upstreams, - self.metrics.clone(), + merge_barrier_align_duration.clone(), ); let actor_id = self.actor_context.id; @@ -189,9 +204,8 @@ impl MergeExecutor { // the one we polled from original upstreams. let mut select_new = SelectReceivers::new( self.actor_context.id, - self.fragment_id, new_upstreams, - self.metrics.clone(), + merge_barrier_align_duration.clone(), ); let new_barrier = expect_first_barrier(&mut select_new).await?; assert_eq!(barrier, &new_barrier); @@ -256,12 +270,10 @@ pub struct SelectReceivers { /// The actor id of this fragment. actor_id: u32, - /// The fragment id - fragment_id: u32, /// watermark column index -> `BufferedWatermarks` buffered_watermarks: BTreeMap>, - /// Streaming Metrics - metrics: Arc, + /// If None, then we don't take `Instant::now()` and `observe` during `poll_next` + merge_barrier_align_duration: Option>, } impl Stream for SelectReceivers { @@ -274,10 +286,6 @@ impl Stream for SelectReceivers { return Poll::Ready(None); } - let merge_barrier_align_duration = self - .metrics - .merge_barrier_align_duration - .with_label_values(&[&self.actor_id.to_string(), &self.fragment_id.to_string()]); let mut start = None; loop { match futures::ready!(self.active.poll_next_unpin(cx)) { @@ -303,7 +311,9 @@ impl Stream for SelectReceivers { } Message::Barrier(barrier) => { // Block this upstream by pushing it to `blocked`. - if self.blocked.is_empty() { + if self.blocked.is_empty() + && self.merge_barrier_align_duration.is_some() + { start = Some(Instant::now()); } self.blocked.push(remaining); @@ -332,7 +342,11 @@ impl Stream for SelectReceivers { Some((None, _)) => unreachable!(), // There's no active upstreams. Process the barrier and resume the blocked ones. None => { - if let Some(start) = start { + if let Some(start) = start + && let Some(merge_barrier_align_duration) = + &self.merge_barrier_align_duration + { + // Observe did a few atomic operation inside, we want to avoid the overhead. merge_barrier_align_duration.observe(start.elapsed().as_secs_f64()) } break; @@ -360,9 +374,8 @@ impl Stream for SelectReceivers { impl SelectReceivers { fn new( actor_id: u32, - fragment_id: u32, upstreams: Vec, - metrics: Arc, + merge_barrier_align_duration: Option>, ) -> Self { assert!(!upstreams.is_empty()); let upstream_actor_ids = upstreams.iter().map(|input| input.actor_id()).collect(); @@ -370,11 +383,10 @@ impl SelectReceivers { blocked: Vec::with_capacity(upstreams.len()), active: Default::default(), actor_id, - fragment_id, barrier: None, upstream_actor_ids, buffered_watermarks: Default::default(), - metrics, + merge_barrier_align_duration, }; this.extend_active(upstreams); this From d488f652ed9d0dbea75a14245b664e0a392d9547 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 12 Jun 2024 18:11:45 +0800 Subject: [PATCH 26/37] feat(storage): avoid uploader depending on seal_epoch (#16985) --- .../hummock_test/src/hummock_storage_tests.rs | 19 +- .../hummock_test/src/state_store_tests.rs | 5 +- .../event_handler/hummock_event_handler.rs | 94 +- src/storage/src/hummock/event_handler/mod.rs | 43 +- .../src/hummock/event_handler/uploader.rs | 901 ++++++++++++------ .../src/hummock/store/hummock_storage.rs | 6 - .../hummock/store/local_hummock_storage.rs | 25 +- src/stream/src/executor/actor.rs | 16 +- .../executor/backfill/arrangement_backfill.rs | 7 +- .../executor/backfill/no_shuffle_backfill.rs | 12 +- 10 files changed, 714 insertions(+), 414 deletions(-) diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 6962fc506ccfb..5c6e1607de926 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -530,6 +530,9 @@ async fn test_state_store_sync() { .await .unwrap(); + let epoch3 = epoch2.next_epoch(); + hummock_storage.seal_current_epoch(epoch3, SealCurrentEpochOptions::for_test()); + let res = test_env.storage.seal_and_sync_epoch(epoch1).await.unwrap(); test_env .meta_client @@ -829,14 +832,15 @@ async fn test_delete_get() { .await .unwrap(); + let epoch2 = epoch1.next_epoch(); + hummock_storage.seal_current_epoch(epoch2, SealCurrentEpochOptions::for_test()); let res = test_env.storage.seal_and_sync_epoch(epoch1).await.unwrap(); test_env .meta_client .commit_epoch(epoch1, res) .await .unwrap(); - let epoch2 = epoch1.next_epoch(); - hummock_storage.seal_current_epoch(epoch2, SealCurrentEpochOptions::for_test()); + let batch2 = vec![( gen_key_from_str(VirtualNode::ZERO, "bb"), StorageValue::new_delete(), @@ -851,6 +855,7 @@ async fn test_delete_get() { ) .await .unwrap(); + hummock_storage.seal_current_epoch(u64::MAX, SealCurrentEpochOptions::for_test()); let res = test_env.storage.seal_and_sync_epoch(epoch2).await.unwrap(); test_env .meta_client @@ -1005,6 +1010,8 @@ async fn test_multiple_epoch_sync() { }; test_get().await; + let epoch4 = epoch3.next_epoch(); + hummock_storage.seal_current_epoch(epoch4, SealCurrentEpochOptions::for_test()); test_env.storage.seal_epoch(epoch1, false); let sync_result2 = test_env.storage.seal_and_sync_epoch(epoch2).await.unwrap(); let sync_result3 = test_env.storage.seal_and_sync_epoch(epoch3).await.unwrap(); @@ -1079,6 +1086,9 @@ async fn test_iter_with_min_epoch() { .await .unwrap(); + let epoch3 = (33 * 1000) << 16; + hummock_storage.seal_current_epoch(epoch3, SealCurrentEpochOptions::for_test()); + { // test before sync { @@ -1329,6 +1339,9 @@ async fn test_hummock_version_reader() { .await .unwrap(); + let epoch4 = (34 * 1000) << 16; + hummock_storage.seal_current_epoch(epoch4, SealCurrentEpochOptions::for_test()); + { // test before sync { @@ -1739,6 +1752,8 @@ async fn test_get_with_min_epoch() { .await .unwrap(); + hummock_storage.seal_current_epoch(u64::MAX, SealCurrentEpochOptions::for_test()); + { // test before sync let k = gen_key(0); diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index 4e14e006f009f..fe129deb52245 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -1046,11 +1046,12 @@ async fn test_delete_get_v2() { ) .await .unwrap(); - let res = hummock_storage.seal_and_sync_epoch(epoch1).await.unwrap(); - meta_client.commit_epoch(epoch1, res).await.unwrap(); let epoch2 = epoch1.next_epoch(); local.seal_current_epoch(epoch2, SealCurrentEpochOptions::for_test()); + let res = hummock_storage.seal_and_sync_epoch(epoch1).await.unwrap(); + meta_client.commit_epoch(epoch1, res).await.unwrap(); + let batch2 = vec![( gen_key_from_str(VirtualNode::ZERO, "bb"), StorageValue::new_delete(), diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index c01a97563237e..277984d3545dd 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -196,7 +196,7 @@ pub struct HummockEventHandler { version_update_rx: UnboundedReceiver, read_version_mapping: Arc>, /// A copy of `read_version_mapping` but owned by event handler - local_read_version_mapping: HashMap, + local_read_version_mapping: HashMap, version_update_notifier_tx: Arc>, pinned_version: Arc>, @@ -455,7 +455,7 @@ impl HummockEventHandler { let mut pending = VecDeque::new(); let mut total_count = 0; for instance_id in instances { - let Some(read_version) = self.local_read_version_mapping.get(&instance_id) else { + let Some((_, read_version)) = self.local_read_version_mapping.get(&instance_id) else { continue; }; total_count += 1; @@ -475,7 +475,7 @@ impl HummockEventHandler { const TRY_LOCK_TIMEOUT: Duration = Duration::from_millis(1); while let Some(instance_id) = pending.pop_front() { - let read_version = self + let (_, read_version) = self .local_read_version_mapping .get(&instance_id) .expect("have checked exist before"); @@ -520,7 +520,6 @@ impl HummockEventHandler { prev_epoch, max_committed_epoch = self.uploader.max_committed_epoch(), max_synced_epoch = self.uploader.max_synced_epoch(), - max_sealed_epoch = self.uploader.max_sealed_epoch(), "handle clear event" ); @@ -588,7 +587,7 @@ impl HummockEventHandler { "read version mapping not empty when clear. remaining tables: {:?}", self.local_read_version_mapping .values() - .map(|read_version| read_version.read().table_id()) + .map(|(_, read_version)| read_version.read().table_id()) .collect_vec() ); @@ -784,6 +783,18 @@ impl HummockEventHandler { HummockEvent::Shutdown => { unreachable!("shutdown is handled specially") } + HummockEvent::InitEpoch { + instance_id, + init_epoch, + } => { + let table_id = self + .local_read_version_mapping + .get(&instance_id) + .expect("should exist") + .0; + self.uploader + .init_instance(instance_id, table_id, init_epoch); + } HummockEvent::ImmToUploader { instance_id, imm } => { assert!( self.local_read_version_mapping.contains_key(&instance_id), @@ -795,29 +806,13 @@ impl HummockEventHandler { self.uploader.may_flush(); } - HummockEvent::SealEpoch { - epoch, - is_checkpoint: _, - } => { - self.uploader.seal_epoch(epoch); - } - HummockEvent::LocalSealEpoch { - epoch, + next_epoch, opts, - table_id, instance_id, } => { - assert!( - self.local_read_version_mapping - .contains_key(&instance_id), - "seal epoch from non-existing read version instance: instance_id: {}, table_id: {}, epoch: {}", - instance_id, table_id, epoch, - ); - if let Some((direction, watermarks)) = opts.table_watermarks { - self.uploader - .add_table_watermarks(epoch, table_id, watermarks, direction) - } + self.uploader + .local_seal_epoch(instance_id, next_epoch, opts); } #[cfg(any(test, feature = "test"))] @@ -852,7 +847,7 @@ impl HummockEventHandler { { self.local_read_version_mapping - .insert(instance_id, basic_read_version.clone()); + .insert(instance_id, (table_id, basic_read_version.clone())); let mut read_version_mapping_guard = self.read_version_mapping.write(); read_version_mapping_guard @@ -876,33 +871,29 @@ impl HummockEventHandler { table_id, instance_id ); guard.event_sender.take().expect("sender is just set"); - self.destroy_read_version(table_id, instance_id); + self.destroy_read_version(instance_id); } } } - HummockEvent::DestroyReadVersion { - table_id, - instance_id, - } => { - self.destroy_read_version(table_id, instance_id); + HummockEvent::DestroyReadVersion { instance_id } => { + self.uploader.may_destroy_instance(instance_id); + self.destroy_read_version(instance_id); } } } - fn destroy_read_version(&mut self, table_id: TableId, instance_id: LocalInstanceId) { + fn destroy_read_version(&mut self, instance_id: LocalInstanceId) { { { - debug!( - "read version deregister: table_id: {}, instance_id: {}", - table_id, instance_id - ); - self.local_read_version_mapping + debug!("read version deregister: instance_id: {}", instance_id); + let (table_id, _) = self + .local_read_version_mapping .remove(&instance_id) .unwrap_or_else(|| { panic!( - "DestroyHummockInstance inexist instance table_id {} instance_id {}", - table_id, instance_id + "DestroyHummockInstance inexist instance instance_id {}", + instance_id ) }); let mut read_version_mapping_guard = self.read_version_mapping.write(); @@ -994,6 +985,7 @@ mod tests { use crate::hummock::test_utils::default_opts_for_test; use crate::hummock::HummockError; use crate::monitor::HummockStateStoreMetrics; + use crate::store::SealCurrentEpochOptions; #[tokio::test] async fn test_clear_shared_buffer() { @@ -1197,6 +1189,11 @@ mod tests { rx.await.unwrap() }; + send_event(HummockEvent::InitEpoch { + instance_id: guard.instance_id, + init_epoch: epoch1, + }); + let imm1 = gen_imm(epoch1).await; read_version .write() @@ -1207,6 +1204,12 @@ mod tests { imm: imm1, }); + send_event(HummockEvent::LocalSealEpoch { + instance_id: guard.instance_id, + next_epoch: epoch2, + opts: SealCurrentEpochOptions::for_test(), + }); + let imm2 = gen_imm(epoch2).await; read_version .write() @@ -1217,20 +1220,19 @@ mod tests { imm: imm2, }); - send_event(HummockEvent::SealEpoch { - epoch: epoch1, - is_checkpoint: true, + let epoch3 = epoch2.next_epoch(); + send_event(HummockEvent::LocalSealEpoch { + instance_id: guard.instance_id, + next_epoch: epoch3, + opts: SealCurrentEpochOptions::for_test(), }); + let (tx1, mut rx1) = oneshot::channel(); send_event(HummockEvent::SyncEpoch { new_sync_epoch: epoch1, sync_result_sender: tx1, }); assert!(poll_fn(|cx| Poll::Ready(rx1.poll_unpin(cx).is_pending())).await); - send_event(HummockEvent::SealEpoch { - epoch: epoch2, - is_checkpoint: true, - }); let (tx2, mut rx2) = oneshot::channel(); send_event(HummockEvent::SyncEpoch { new_sync_epoch: epoch2, diff --git a/src/storage/src/hummock/event_handler/mod.rs b/src/storage/src/hummock/event_handler/mod.rs index c28dd6d25c3a4..bbf69ae194f72 100644 --- a/src/storage/src/hummock/event_handler/mod.rs +++ b/src/storage/src/hummock/event_handler/mod.rs @@ -72,15 +72,14 @@ pub enum HummockEvent { imm: ImmutableMemtable, }, - SealEpoch { - epoch: HummockEpoch, - is_checkpoint: bool, + InitEpoch { + instance_id: LocalInstanceId, + init_epoch: HummockEpoch, }, LocalSealEpoch { instance_id: LocalInstanceId, - table_id: TableId, - epoch: HummockEpoch, + next_epoch: HummockEpoch, opts: SealCurrentEpochOptions, }, @@ -97,7 +96,6 @@ pub enum HummockEvent { }, DestroyReadVersion { - table_id: TableId, instance_id: LocalInstanceId, }, } @@ -116,27 +114,25 @@ impl HummockEvent { HummockEvent::Shutdown => "Shutdown".to_string(), + HummockEvent::InitEpoch { + instance_id, + init_epoch, + } => { + format!("InitEpoch {} {}", instance_id, init_epoch) + } + HummockEvent::ImmToUploader { instance_id, imm } => { format!("ImmToUploader {} {}", instance_id, imm.batch_id()) } - HummockEvent::SealEpoch { - epoch, - is_checkpoint, - } => format!( - "SealEpoch epoch {:?} is_checkpoint {:?}", - epoch, is_checkpoint - ), - HummockEvent::LocalSealEpoch { - epoch, instance_id, - table_id, + next_epoch, opts, } => { format!( - "LocalSealEpoch epoch: {}, table_id: {}, instance_id: {}, opts: {:?}", - epoch, table_id.table_id, instance_id, opts + "LocalSealEpoch next_epoch: {}, instance_id: {}, opts: {:?}", + next_epoch, instance_id, opts ) } @@ -150,13 +146,9 @@ impl HummockEvent { table_id, is_replicated ), - HummockEvent::DestroyReadVersion { - table_id, - instance_id, - } => format!( - "DestroyReadVersion table_id {:?} instance_id {:?}", - table_id, instance_id - ), + HummockEvent::DestroyReadVersion { instance_id } => { + format!("DestroyReadVersion instance_id {:?}", instance_id) + } #[cfg(any(test, feature = "test"))] HummockEvent::FlushEvent(_) => "FlushEvent".to_string(), @@ -210,7 +202,6 @@ impl Drop for LocalInstanceGuard { // need to handle failure sender .send(HummockEvent::DestroyReadVersion { - table_id: self.table_id, instance_id: self.instance_id, }) .unwrap_or_else(|err| { diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index 1feaf017b2f02..f768aa23dcd89 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::hash_map::Entry; +use std::cmp::Ordering; +use std::collections::btree_map::Entry; use std::collections::{BTreeMap, HashMap, VecDeque}; use std::fmt::{Debug, Display, Formatter}; use std::future::{poll_fn, Future}; -use std::mem::{replace, take}; +use std::mem::{replace, swap, take}; use std::pin::Pin; use std::sync::Arc; use std::task::{ready, Context, Poll}; @@ -38,7 +39,7 @@ use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, LocalSstableInfo, use thiserror_ext::AsReport; use tokio::sync::oneshot; use tokio::task::JoinHandle; -use tracing::{debug, error, info}; +use tracing::{debug, error, info, warn}; use crate::hummock::event_handler::hummock_event_handler::{send_sync_result, BufferTracker}; use crate::hummock::event_handler::uploader::uploader_imm::UploaderImm; @@ -50,6 +51,17 @@ use crate::hummock::{HummockError, HummockResult, ImmutableMemtable}; use crate::mem_table::ImmId; use crate::monitor::HummockStateStoreMetrics; use crate::opts::StorageOpts; +use crate::store::SealCurrentEpochOptions; + +/// Take epoch data inclusively before `epoch` out from `data` +fn take_before_epoch( + data: &mut BTreeMap, + epoch: HummockEpoch, +) -> BTreeMap { + let mut before_epoch_data = data.split_off(&(epoch + 1)); + swap(&mut before_epoch_data, data); + before_epoch_data +} type UploadTaskInput = HashMap>; pub type UploadTaskPayload = HashMap>; @@ -329,11 +341,6 @@ struct SpilledData { } impl SpilledData { - #[cfg(test)] - fn is_empty(&self) -> bool { - self.uploading_tasks.is_empty() && self.uploaded_data.is_empty() - } - fn add_task(&mut self, task: UploadingTask) { self.uploading_tasks.push_front(task); } @@ -360,20 +367,16 @@ impl SpilledData { } #[derive(Default, Debug)] -struct UnsealedEpochData { - // newer data at the front - imms: HashMap>, +struct EpochData { spilled_data: SpilledData, - - table_watermarks: HashMap, BitmapBuilder)>, } -impl UnsealedEpochData { - fn flush(&mut self, context: &UploaderContext) -> usize { - let imms: HashMap<_, _> = take(&mut self.imms) - .into_iter() - .map(|(id, imms)| (id, imms.into_iter().collect_vec())) - .collect(); +impl EpochData { + fn flush( + &mut self, + context: &UploaderContext, + imms: HashMap>, + ) -> usize { if !imms.is_empty() { let task = UploadingTask::new(imms, context); context.stats.spill_task_counts_from_unsealed.inc(); @@ -389,10 +392,12 @@ impl UnsealedEpochData { 0 } } +} +impl TableUnsyncData { fn add_table_watermarks( &mut self, - table_id: TableId, + epoch: HummockEpoch, table_watermarks: Vec, direction: WatermarkDirection, ) { @@ -411,45 +416,50 @@ impl UnsealedEpochData { } } } - match self.table_watermarks.entry(table_id) { - Entry::Occupied(mut entry) => { - let (prev_direction, prev_watermarks, vnode_bitmap) = entry.get_mut(); + match &mut self.table_watermarks { + Some((prev_direction, prev_watermarks)) => { assert_eq!( *prev_direction, direction, "table id {} new watermark direction not match with previous", - table_id + self.table_id ); - apply_new_vnodes(vnode_bitmap, &table_watermarks); - prev_watermarks.extend(table_watermarks); + match prev_watermarks.entry(epoch) { + Entry::Occupied(mut entry) => { + let (prev_watermarks, vnode_bitmap) = entry.get_mut(); + apply_new_vnodes(vnode_bitmap, &table_watermarks); + prev_watermarks.extend(table_watermarks); + } + Entry::Vacant(entry) => { + let mut vnode_bitmap = BitmapBuilder::zeroed(VirtualNode::COUNT); + apply_new_vnodes(&mut vnode_bitmap, &table_watermarks); + entry.insert((table_watermarks, vnode_bitmap)); + } + } } - Entry::Vacant(entry) => { + None => { let mut vnode_bitmap = BitmapBuilder::zeroed(VirtualNode::COUNT); apply_new_vnodes(&mut vnode_bitmap, &table_watermarks); - entry.insert((direction, table_watermarks, vnode_bitmap)); + self.table_watermarks = Some(( + direction, + BTreeMap::from_iter([(epoch, (table_watermarks, vnode_bitmap))]), + )); } } } } #[derive(Default)] -/// Data at the sealed stage. We will ensure that data in `imms` are newer than the data in the -/// `spilled_data`, and that data in the `uploading_tasks` in `spilled_data` are newer than data in -/// the `uploaded_data` in `spilled_data`. -struct SealedData { +struct SyncDataBuilder { // newer epochs come first epochs: VecDeque, - // Sealed imms grouped by table shard. - // newer data (larger imm id) at the front - imms_by_table_shard: HashMap>, - spilled_data: SpilledData, table_watermarks: HashMap, } -impl SealedData { - /// Add the data of a newly sealed epoch. +impl SyncDataBuilder { + /// Add the data of a new epoch. /// /// Note: it may happen that, for example, currently we hold `imms` and `spilled_data` of epoch /// 3, and after we add the spilled data of epoch 4, both `imms` and `spilled_data` hold data @@ -459,9 +469,9 @@ impl SealedData { /// data of `imms` must not overlap with the epoch 4 data of `spilled_data`. The explanation is /// as followed: /// - /// First, unsealed data has 3 stages, from earlier to later, imms, uploading task, and - /// uploaded. When we try to spill unsealed data, we first pick the imms of older epoch until - /// the imms of older epoch are all picked. When we try to poll the uploading tasks of unsealed + /// First, unsync data has 3 stages, from earlier to later, imms, uploading task, and + /// uploaded. When we try to spill unsync data, we first pick the imms of older epoch until + /// the imms of older epoch are all picked. When we try to poll the uploading tasks of unsync /// data, we first poll the task of older epoch, until there is no uploading task in older /// epoch. Therefore, we can reach that, if two data are in the same stage, but /// different epochs, data in the older epoch will always enter the next stage earlier than data @@ -475,28 +485,19 @@ impl SealedData { /// Based on the two points above, we can reach that, if two data of a same key appear in /// different epochs, the data of older epoch will not appear at a later stage than the data /// of newer epoch. Therefore, we can safely merge the data of each stage when we seal an epoch. - fn seal_new_epoch(&mut self, epoch: HummockEpoch, mut unseal_epoch_data: UnsealedEpochData) { - if let Some(prev_max_sealed_epoch) = self.epochs.front() { + fn add_new_epoch(&mut self, epoch: HummockEpoch, mut unseal_epoch_data: EpochData) { + if let Some(prev_max_epoch) = self.epochs.front() { assert!( - epoch > *prev_max_sealed_epoch, - "epoch {} to seal not greater than prev max sealed epoch {}", + epoch > *prev_max_epoch, + "epoch {} to seal not greater than prev max epoch {}", epoch, - prev_max_sealed_epoch + prev_max_epoch ); } - // rearrange sealed imms by table shard and in epoch descending order - for (instance_id, imms) in unseal_epoch_data.imms { - let queue = self.imms_by_table_shard.entry(instance_id).or_default(); - for imm in imms.into_iter().rev() { - if let Some(front) = queue.front() { - assert_gt!(imm.batch_id(), front.batch_id()); - } - queue.push_front(imm); - } - } - self.epochs.push_front(epoch); + // for each local instance, earlier data must be spilled at earlier epoch. Therefore, since we add spill data from old epoch + // to new epoch, unseal_epoch_data .spilled_data .uploading_tasks @@ -507,64 +508,403 @@ impl SealedData { .append(&mut self.spilled_data.uploaded_data); self.spilled_data.uploading_tasks = unseal_epoch_data.spilled_data.uploading_tasks; self.spilled_data.uploaded_data = unseal_epoch_data.spilled_data.uploaded_data; - for (table_id, (direction, watermarks, _)) in unseal_epoch_data.table_watermarks { - match self.table_watermarks.entry(table_id) { - Entry::Occupied(mut entry) => { - entry.get_mut().add_new_epoch_watermarks( + } + + fn add_table_watermarks( + &mut self, + table_id: TableId, + direction: WatermarkDirection, + watermarks: impl Iterator)>, + ) { + let mut table_watermarks: Option = None; + for (epoch, watermarks) in watermarks { + match &mut table_watermarks { + Some(prev_watermarks) => { + prev_watermarks.add_new_epoch_watermarks( epoch, Arc::from(watermarks), direction, ); } - Entry::Vacant(entry) => { - entry.insert(TableWatermarks::single_epoch(epoch, watermarks, direction)); + None => { + table_watermarks = + Some(TableWatermarks::single_epoch(epoch, watermarks, direction)); } - }; + } + } + if let Some(table_watermarks) = table_watermarks { + assert!(self + .table_watermarks + .insert(table_id, table_watermarks) + .is_none()); } } - // Flush can be triggered by either a sync_epoch or a spill (`may_flush`) request. - fn flush(&mut self, context: &UploaderContext, is_spilled: bool) -> usize { - let payload: HashMap<_, Vec<_>> = take(&mut self.imms_by_table_shard) - .into_iter() - .map(|(id, imms)| (id, imms.into_iter().collect())) - .collect(); - + fn flush(&mut self, context: &UploaderContext, payload: UploadTaskInput) { if !payload.is_empty() { let task = UploadingTask::new(payload, context); - let size = task.task_info.task_size; - if is_spilled { - context.stats.spill_task_counts_from_sealed.inc(); - context - .stats - .spill_task_size_from_sealed - .inc_by(task.task_info.task_size as u64); - info!("Spill sealed data. Task: {}", task.get_task_info()); - } self.spilled_data.add_task(task); - size + } + } +} + +struct LocalInstanceEpochData { + epoch: HummockEpoch, + // newer data comes first. + imms: VecDeque, + has_spilled: bool, +} + +impl LocalInstanceEpochData { + fn new(epoch: HummockEpoch) -> Self { + Self { + epoch, + imms: VecDeque::new(), + has_spilled: false, + } + } + + fn epoch(&self) -> HummockEpoch { + self.epoch + } + + fn add_imm(&mut self, imm: UploaderImm) { + assert_eq!(imm.max_epoch(), imm.min_epoch()); + assert_eq!(self.epoch, imm.min_epoch()); + if let Some(prev_imm) = self.imms.front() { + assert_gt!(imm.batch_id(), prev_imm.batch_id()); + } + self.imms.push_front(imm); + } + + fn is_empty(&self) -> bool { + self.imms.is_empty() + } +} + +struct LocalInstanceUnsyncData { + table_id: TableId, + instance_id: LocalInstanceId, + // None means that the current instance should have stopped advancing + current_epoch_data: Option, + // newer data comes first. + sealed_data: VecDeque, + // newer data comes first + flushing_imms: VecDeque, +} + +impl LocalInstanceUnsyncData { + fn new(table_id: TableId, instance_id: LocalInstanceId, init_epoch: HummockEpoch) -> Self { + Self { + table_id, + instance_id, + current_epoch_data: Some(LocalInstanceEpochData::new(init_epoch)), + sealed_data: VecDeque::new(), + flushing_imms: Default::default(), + } + } + + fn add_imm(&mut self, imm: UploaderImm) { + assert_eq!(self.table_id, imm.table_id); + self.current_epoch_data + .as_mut() + .expect("should be Some when adding new imm") + .add_imm(imm); + } + + fn local_seal_epoch(&mut self, next_epoch: HummockEpoch) -> HummockEpoch { + let data = self + .current_epoch_data + .as_mut() + .expect("should be Some when seal new epoch"); + let current_epoch = data.epoch; + debug!( + instance_id = self.instance_id, + next_epoch, current_epoch, "local seal epoch" + ); + assert_gt!(next_epoch, current_epoch); + let epoch_data = replace(data, LocalInstanceEpochData::new(next_epoch)); + if !epoch_data.is_empty() { + self.sealed_data.push_front(epoch_data); + } + current_epoch + } + + // imm_ids from old to new, which means in ascending order + fn ack_flushed(&mut self, imm_ids: impl Iterator) { + for imm_id in imm_ids { + assert_eq!(self.flushing_imms.pop_back().expect("should exist"), imm_id); + } + } + + fn spill(&mut self, epoch: HummockEpoch) -> Vec { + let imms = if let Some(oldest_sealed_epoch) = self.sealed_data.back() { + match oldest_sealed_epoch.epoch.cmp(&epoch) { + Ordering::Less => { + unreachable!( + "should not spill at this epoch because there \ + is unspilled data in previous epoch: prev epoch {}, spill epoch {}", + oldest_sealed_epoch.epoch, epoch + ); + } + Ordering::Equal => { + let epoch_data = self.sealed_data.pop_back().unwrap(); + assert_eq!(epoch, epoch_data.epoch); + epoch_data.imms + } + Ordering::Greater => VecDeque::new(), + } } else { - 0 + let Some(current_epoch_data) = &mut self.current_epoch_data else { + return Vec::new(); + }; + match current_epoch_data.epoch.cmp(&epoch) { + Ordering::Less => { + assert!( + current_epoch_data.imms.is_empty(), + "should not spill at this epoch because there \ + is unspilled data in current epoch epoch {}, spill epoch {}", + current_epoch_data.epoch, + epoch + ); + VecDeque::new() + } + Ordering::Equal => { + if !current_epoch_data.imms.is_empty() { + current_epoch_data.has_spilled = true; + take(&mut current_epoch_data.imms) + } else { + VecDeque::new() + } + } + Ordering::Greater => VecDeque::new(), + } + }; + self.add_flushing_imm(imms.iter().rev().map(|imm| imm.batch_id())); + imms.into_iter().collect() + } + + fn add_flushing_imm(&mut self, imm_ids: impl Iterator) { + for imm_id in imm_ids { + if let Some(prev_imm_id) = self.flushing_imms.front() { + assert_gt!(imm_id, *prev_imm_id); + } + self.flushing_imms.push_front(imm_id); } } - /// Clear self and return the current sealed data - fn drain(&mut self) -> SealedData { - take(self) + // start syncing the imm inclusively before the `epoch` + // returning data with newer data coming first + fn sync(&mut self, epoch: HummockEpoch) -> Vec { + // firstly added from old to new + let mut ret = Vec::new(); + while let Some(epoch_data) = self.sealed_data.back() + && epoch_data.epoch() <= epoch + { + let imms = self.sealed_data.pop_back().expect("checked exist").imms; + self.add_flushing_imm(imms.iter().rev().map(|imm| imm.batch_id())); + ret.extend(imms.into_iter().rev()); + } + // reverse so that newer data comes first + ret.reverse(); + if let Some(latest_epoch_data) = &self.current_epoch_data { + if latest_epoch_data.epoch <= epoch { + assert!(self.sealed_data.is_empty()); + assert!(latest_epoch_data.is_empty()); + assert!(!latest_epoch_data.has_spilled); + if cfg!(debug_assertions) { + panic!("sync epoch exceeds latest epoch, and the current instance should have be archived"); + } + warn!( + instance_id = self.instance_id, + table_id = self.table_id.table_id, + "sync epoch exceeds latest epoch, and the current instance should have be archived" + ); + self.current_epoch_data = None; + } + } + ret } +} - #[cfg(test)] - fn imm_count(&self) -> usize { - self.imms_by_table_shard - .values() - .map(|imms| imms.len()) - .sum() +struct TableUnsyncData { + table_id: TableId, + instance_data: HashMap, + #[expect(clippy::type_complexity)] + table_watermarks: Option<( + WatermarkDirection, + BTreeMap, BitmapBuilder)>, + )>, +} + +impl TableUnsyncData { + fn new(table_id: TableId) -> Self { + Self { + table_id, + instance_data: Default::default(), + table_watermarks: None, + } + } + + fn sync( + &mut self, + epoch: HummockEpoch, + ) -> ( + impl Iterator)> + '_, + Option<( + WatermarkDirection, + impl Iterator)>, + )>, + ) { + ( + self.instance_data + .iter_mut() + .map(move |(instance_id, data)| (*instance_id, data.sync(epoch))), + self.table_watermarks + .as_mut() + .map(|(direction, watermarks)| { + let watermarks = take_before_epoch(watermarks, epoch); + ( + *direction, + watermarks + .into_iter() + .map(|(epoch, (watermarks, _))| (epoch, watermarks)), + ) + }), + ) + } +} + +#[derive(Default)] +/// Unsync data, can be either imm or spilled sst, and some aggregated epoch information. +/// +/// `instance_data` holds the imm of each individual local instance, and data are first added here. +/// The aggregated epoch information (table watermarks, etc.) and the spilled sst will be added to `epoch_data`. +struct UnsyncData { + table_data: HashMap, + // An index as a mapping from instance id to its table id + instance_table_id: HashMap, + epoch_data: BTreeMap, +} + +impl UnsyncData { + fn init_instance( + &mut self, + table_id: TableId, + instance_id: LocalInstanceId, + init_epoch: HummockEpoch, + ) { + debug!( + table_id = table_id.table_id, + instance_id, init_epoch, "init epoch" + ); + let table_data = self + .table_data + .entry(table_id) + .or_insert_with(|| TableUnsyncData::new(table_id)); + assert!(table_data + .instance_data + .insert( + instance_id, + LocalInstanceUnsyncData::new(table_id, instance_id, init_epoch) + ) + .is_none()); + assert!(self + .instance_table_id + .insert(instance_id, table_id) + .is_none()); + self.epoch_data.entry(init_epoch).or_default(); + } + + fn instance_data( + &mut self, + instance_id: LocalInstanceId, + ) -> Option<&mut LocalInstanceUnsyncData> { + self.instance_table_id + .get_mut(&instance_id) + .cloned() + .map(move |table_id| { + self.table_data + .get_mut(&table_id) + .expect("should exist") + .instance_data + .get_mut(&instance_id) + .expect("should exist") + }) + } + + fn add_imm(&mut self, instance_id: LocalInstanceId, imm: UploaderImm) { + self.instance_data(instance_id) + .expect("should exist") + .add_imm(imm); + } + + fn local_seal_epoch( + &mut self, + instance_id: LocalInstanceId, + next_epoch: HummockEpoch, + opts: SealCurrentEpochOptions, + ) { + let table_id = self.instance_table_id[&instance_id]; + let table_data = self.table_data.get_mut(&table_id).expect("should exist"); + let instance_data = table_data + .instance_data + .get_mut(&instance_id) + .expect("should exist"); + let epoch = instance_data.local_seal_epoch(next_epoch); + self.epoch_data.entry(next_epoch).or_default(); + if let Some((direction, table_watermarks)) = opts.table_watermarks { + table_data.add_table_watermarks(epoch, table_watermarks, direction); + } + } + + fn may_destroy_instance(&mut self, instance_id: LocalInstanceId) { + if let Some(table_id) = self.instance_table_id.remove(&instance_id) { + debug!(instance_id, "destroy instance"); + let table_data = self.table_data.get_mut(&table_id).expect("should exist"); + assert!(table_data.instance_data.remove(&instance_id).is_some()); + if table_data.instance_data.is_empty() { + self.table_data.remove(&table_id); + } + } + } + + fn sync(&mut self, epoch: HummockEpoch, context: &UploaderContext) -> SyncDataBuilder { + let sync_epoch_data = take_before_epoch(&mut self.epoch_data, epoch); + + let mut sync_data = SyncDataBuilder::default(); + for (epoch, epoch_data) in sync_epoch_data { + sync_data.add_new_epoch(epoch, epoch_data); + } + + let mut flush_payload = HashMap::new(); + for (table_id, table_data) in &mut self.table_data { + let (unflushed_payload, table_watermarks) = table_data.sync(epoch); + for (instance_id, payload) in unflushed_payload { + if !payload.is_empty() { + flush_payload.insert(instance_id, payload); + } + } + if let Some((direction, watermarks)) = table_watermarks { + sync_data.add_table_watermarks(*table_id, direction, watermarks); + } + } + sync_data.flush(context, flush_payload); + sync_data + } + + fn ack_flushed(&mut self, sstable_info: &StagingSstableInfo) { + for (instance_id, imm_ids) in sstable_info.imm_ids() { + if let Some(instance_data) = self.instance_data(*instance_id) { + // take `rev` to let old imm id goes first + instance_data.ack_flushed(imm_ids.iter().rev().cloned()); + } + } } } struct SyncingData { - // newer epochs come first - epochs: Vec, + sync_epoch: HummockEpoch, // TODO: may replace `TryJoinAll` with a future that will abort other join handles once // one join handle failed. // None means there is no pending uploading tasks @@ -575,13 +915,7 @@ struct SyncingData { sync_result_sender: oneshot::Sender>, } -impl SyncingData { - fn sync_epoch(&self) -> HummockEpoch { - *self.epochs.first().expect("non-empty") - } -} - -pub(super) struct SyncedData { +pub struct SyncedData { pub newly_upload_ssts: Vec, pub uploaded_ssts: VecDeque, pub table_watermarks: HashMap, @@ -615,12 +949,7 @@ impl UploaderContext { #[derive(Default)] struct UploaderData { - /// Data that are not sealed yet. `epoch` satisfies `epoch > max_sealed_epoch`. - unsealed_data: BTreeMap, - - /// Data that are sealed but not synced yet. `epoch` satisfies - /// `max_syncing_epoch < epoch <= max_sealed_epoch`. - sealed_data: SealedData, + unsync_data: UnsyncData, /// Data that has started syncing but not synced yet. `epoch` satisfies /// `max_synced_epoch < epoch <= max_syncing_epoch`. @@ -630,9 +959,8 @@ struct UploaderData { impl UploaderData { fn abort(self, err: impl Fn() -> HummockError) { - self.sealed_data.spilled_data.abort(); - for (_, unsealed_data) in self.unsealed_data { - unsealed_data.spilled_data.abort(); + for (_, epoch_data) in self.unsync_data.epoch_data { + epoch_data.spilled_data.abort(); } // TODO: call `abort` on the uploading task join handle of syncing_data for syncing_data in self.syncing_data { @@ -651,20 +979,17 @@ enum UploaderState { /// An uploader for hummock data. /// -/// Data have 4 sequential stages: unsealed, sealed, syncing, synced. +/// Data have 3 sequential stages: unsync (inside each local instance, data can be unsealed, sealed), syncing, synced. /// -/// The 4 stages are divided by 3 marginal epochs: `max_sealed_epoch`, `max_syncing_epoch`, +/// The 3 stages are divided by 2 marginal epochs: `max_syncing_epoch`, /// `max_synced_epoch`. Epochs satisfy the following inequality. /// /// (epochs of `synced_data`) <= `max_synced_epoch` < (epochs of `syncing_data`) <= -/// `max_syncing_epoch` < (epochs of `sealed_data`) <= `max_sealed_epoch` < (epochs of -/// `unsealed_data`) +/// `max_syncing_epoch` < (epochs of `unsync_data`) /// /// Data are mostly stored in `VecDeque`, and the order stored in the `VecDeque` indicates the data /// order. Data at the front represents ***newer*** data. pub struct HummockUploader { - /// The maximum epoch that is sealed - max_sealed_epoch: HummockEpoch, /// The maximum epoch that has started syncing max_syncing_epoch: HummockEpoch, /// The maximum epoch that has been synced @@ -685,12 +1010,10 @@ impl HummockUploader { ) -> Self { let initial_epoch = pinned_version.version().max_committed_epoch; Self { - max_sealed_epoch: initial_epoch, max_syncing_epoch: initial_epoch, max_synced_epoch: initial_epoch, state: UploaderState::Working(UploaderData { - unsealed_data: Default::default(), - sealed_data: Default::default(), + unsync_data: Default::default(), syncing_data: Default::default(), }), context: UploaderContext::new( @@ -707,10 +1030,6 @@ impl HummockUploader { &self.context.buffer_tracker } - pub(super) fn max_sealed_epoch(&self) -> HummockEpoch { - self.max_sealed_epoch - } - pub(super) fn max_synced_epoch(&self) -> HummockEpoch { self.max_synced_epoch } @@ -727,78 +1046,36 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return; }; - let epoch = imm.min_epoch(); - assert!( - epoch > self.max_sealed_epoch, - "imm epoch {} older than max sealed epoch {}", - epoch, - self.max_sealed_epoch - ); - let unsealed_data = data.unsealed_data.entry(epoch).or_default(); - unsealed_data - .imms - .entry(instance_id) - .or_default() - .push_front(UploaderImm::new(imm, &self.context)); + let imm = UploaderImm::new(imm, &self.context); + data.unsync_data.add_imm(instance_id, imm); } - pub(super) fn add_table_watermarks( + pub(super) fn init_instance( &mut self, - epoch: u64, + instance_id: LocalInstanceId, table_id: TableId, - table_watermarks: Vec, - direction: WatermarkDirection, + init_epoch: HummockEpoch, ) { let UploaderState::Working(data) = &mut self.state else { return; }; - assert!( - epoch > self.max_sealed_epoch, - "imm epoch {} older than max sealed epoch {}", - epoch, - self.max_sealed_epoch - ); - data.unsealed_data - .entry(epoch) - .or_default() - .add_table_watermarks(table_id, table_watermarks, direction); + assert_gt!(init_epoch, self.max_syncing_epoch); + data.unsync_data + .init_instance(table_id, instance_id, init_epoch); } - pub(super) fn seal_epoch(&mut self, epoch: HummockEpoch) { + pub(super) fn local_seal_epoch( + &mut self, + instance_id: LocalInstanceId, + next_epoch: HummockEpoch, + opts: SealCurrentEpochOptions, + ) { let UploaderState::Working(data) = &mut self.state else { return; }; - debug!("epoch {} is sealed", epoch); - assert!( - epoch > self.max_sealed_epoch, - "sealing a sealed epoch {}. {}", - epoch, - self.max_sealed_epoch - ); - self.max_sealed_epoch = epoch; - let unsealed_data = - if let Some((&smallest_unsealed_epoch, _)) = data.unsealed_data.first_key_value() { - assert!( - smallest_unsealed_epoch >= epoch, - "some epoch {} older than epoch to seal {}", - smallest_unsealed_epoch, - epoch - ); - if smallest_unsealed_epoch == epoch { - let (_, unsealed_data) = data - .unsealed_data - .pop_first() - .expect("we have checked non-empty"); - unsealed_data - } else { - debug!("epoch {} to seal has no data", epoch); - UnsealedEpochData::default() - } - } else { - debug!("epoch {} to seal has no data", epoch); - UnsealedEpochData::default() - }; - data.sealed_data.seal_new_epoch(epoch, unsealed_data); + assert_gt!(next_epoch, self.max_syncing_epoch); + data.unsync_data + .local_seal_epoch(instance_id, next_epoch, opts); } pub(super) fn start_sync_epoch( @@ -827,20 +1104,12 @@ impl HummockUploader { epoch, self.max_syncing_epoch ); - assert_eq!( - epoch, self.max_sealed_epoch, - "we must start syncing all the sealed data", - ); self.max_syncing_epoch = epoch; - // flush imms to SST file, the output SSTs will be uploaded to object store - // return unfinished merging task - data.sealed_data.flush(&self.context, false); + let sync_data = data.unsync_data.sync(epoch, &self.context); - let SealedData { - epochs, - imms_by_table_shard, + let SyncDataBuilder { spilled_data: SpilledData { uploading_tasks, @@ -848,14 +1117,7 @@ impl HummockUploader { }, table_watermarks, .. - } = data.sealed_data.drain(); - - assert!( - imms_by_table_shard.is_empty(), - "after flush, imms must be empty" - ); - - assert_eq!(epoch, *epochs.front().expect("non-empty epoch")); + } = sync_data; let try_join_all_upload_task = if uploading_tasks.is_empty() { None @@ -864,7 +1126,7 @@ impl HummockUploader { }; data.syncing_data.push_front(SyncingData { - epochs: epochs.into_iter().collect(), + sync_epoch: epoch, uploading_tasks: try_join_all_upload_task, uploaded: uploaded_data, table_watermarks, @@ -902,34 +1164,21 @@ impl HummockUploader { self.context.pinned_version = pinned_version; if self.max_synced_epoch < max_committed_epoch { self.max_synced_epoch = max_committed_epoch; - if let UploaderState::Working(data) = &mut self.state { - if let Some(syncing_data) = data.syncing_data.back() { - // there must not be any syncing data below MCE - assert_gt!( - *syncing_data - .epochs - .last() - .expect("epoch should not be empty"), - max_committed_epoch - ); - } - }; } if self.max_syncing_epoch < max_committed_epoch { self.max_syncing_epoch = max_committed_epoch; - if let UploaderState::Working(data) = &mut self.state { - // there must not be any sealed data below MCE - if let Some(&epoch) = data.sealed_data.epochs.back() { - assert_gt!(epoch, max_committed_epoch); - } - } - } - if self.max_sealed_epoch < max_committed_epoch { - self.max_sealed_epoch = max_committed_epoch; - if let UploaderState::Working(data) = &mut self.state { - // there must not be any unsealed data below MCE - if let Some((&epoch, _)) = data.unsealed_data.first_key_value() { - assert_gt!(epoch, max_committed_epoch); + if let UploaderState::Working(data) = &self.state { + for instance_data in data + .unsync_data + .table_data + .values() + .flat_map(|data| data.instance_data.values()) + { + if let Some(oldest_epoch) = instance_data.sealed_data.back() { + assert_gt!(oldest_epoch.epoch, max_committed_epoch); + } else if let Some(current_epoch) = &instance_data.current_epoch_data { + assert_gt!(current_epoch.epoch, max_committed_epoch); + } } } } @@ -941,26 +1190,28 @@ impl HummockUploader { }; if self.context.buffer_tracker.need_flush() { let mut curr_batch_flush_size = 0; - if self.context.buffer_tracker.need_flush() { - curr_batch_flush_size += data.sealed_data.flush(&self.context, true); - } - - if self - .context - .buffer_tracker - .need_more_flush(curr_batch_flush_size) - { - // iterate from older epoch to newer epoch - for unsealed_data in data.unsealed_data.values_mut() { - curr_batch_flush_size += unsealed_data.flush(&self.context); - if !self - .context - .buffer_tracker - .need_more_flush(curr_batch_flush_size) - { - break; + // iterate from older epoch to newer epoch + for (epoch, epoch_data) in &mut data.unsync_data.epoch_data { + if !self + .context + .buffer_tracker + .need_more_flush(curr_batch_flush_size) + { + break; + } + let mut payload = HashMap::new(); + for (instance_id, instance_data) in data + .unsync_data + .table_data + .values_mut() + .flat_map(|data| data.instance_data.iter_mut()) + { + let instance_payload = instance_data.spill(*epoch); + if !instance_payload.is_empty() { + payload.insert(*instance_id, instance_payload); } } + curr_batch_flush_size += epoch_data.flush(&self.context, payload); } curr_batch_flush_size > 0 } else { @@ -972,7 +1223,6 @@ impl HummockUploader { let max_committed_epoch = self.context.pinned_version.max_committed_epoch(); self.max_synced_epoch = max_committed_epoch; self.max_syncing_epoch = max_committed_epoch; - self.max_sealed_epoch = max_committed_epoch; if let UploaderState::Working(data) = replace( &mut self.state, UploaderState::Working(UploaderData::default()), @@ -984,6 +1234,13 @@ impl HummockUploader { self.context.stats.uploader_syncing_epoch_count.set(0); } + + pub(crate) fn may_destroy_instance(&mut self, instance_id: LocalInstanceId) { + let UploaderState::Working(data) = &mut self.state else { + return; + }; + data.unsync_data.may_destroy_instance(instance_id); + } } impl UploaderData { @@ -1015,12 +1272,18 @@ impl UploaderData { .stats .uploader_syncing_epoch_count .set(self.syncing_data.len() as _); - let epoch = syncing_data.sync_epoch(); + let epoch = syncing_data.sync_epoch; - let result = result.map(|newly_uploaded_sstable_infos| SyncedData { - newly_upload_ssts: newly_uploaded_sstable_infos, - uploaded_ssts: syncing_data.uploaded, - table_watermarks: syncing_data.table_watermarks, + let result = result.map(|newly_uploaded_sstable_infos| { + // take `rev` so that old data is acked first + for sstable_info in newly_uploaded_sstable_infos.iter().rev() { + self.unsync_data.ack_flushed(sstable_info); + } + SyncedData { + newly_upload_ssts: newly_uploaded_sstable_infos, + uploaded_ssts: syncing_data.uploaded, + table_watermarks: syncing_data.table_watermarks, + } }); Poll::Ready(Some((epoch, result, syncing_data.sync_result_sender))) @@ -1029,23 +1292,15 @@ impl UploaderData { } } - /// Poll the success of the oldest spilled task of sealed data. Return `Poll::Ready(None)` if - /// there is no spilling task. - fn poll_sealed_spill_task(&mut self, cx: &mut Context<'_>) -> Poll> { - self.sealed_data.spilled_data.poll_success_spill(cx) - } - - /// Poll the success of the oldest spilled task of unsealed data. Return `Poll::Ready(None)` if + /// Poll the success of the oldest spilled task of unsync spill data. Return `Poll::Ready(None)` if /// there is no spilling task. - fn poll_unsealed_spill_task( - &mut self, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_spill_task(&mut self, cx: &mut Context<'_>) -> Poll> { // iterator from older epoch to new epoch so that the spill task are finished in epoch order - for unsealed_data in self.unsealed_data.values_mut() { - // if None, there is no spilling task. Search for the unsealed data of the next epoch in + for epoch_data in self.unsync_data.epoch_data.values_mut() { + // if None, there is no spilling task. Search for the unsync data of the next epoch in // the next iteration. - if let Some(sstable_info) = ready!(unsealed_data.spilled_data.poll_success_spill(cx)) { + if let Some(sstable_info) = ready!(epoch_data.spilled_data.poll_success_spill(cx)) { + self.unsync_data.ack_flushed(&sstable_info); return Poll::Ready(Some(sstable_info)); } } @@ -1093,17 +1348,12 @@ impl HummockUploader { data.abort(|| { HummockError::other(format!("previous epoch {} failed to sync", epoch)) }); - return Poll::Pending; } - } - } - - if let Some(sstable_info) = ready!(data.poll_sealed_spill_task(cx)) { - return Poll::Ready(UploaderEvent::DataSpilled(sstable_info)); + }; } - if let Some(sstable_info) = ready!(data.poll_unsealed_spill_task(cx)) { + if let Some(sstable_info) = ready!(data.poll_spill_task(cx)) { return Poll::Ready(UploaderEvent::DataSpilled(sstable_info)); } @@ -1156,6 +1406,7 @@ pub(crate) mod tests { use crate::mem_table::{ImmId, ImmutableMemtable}; use crate::monitor::HummockStateStoreMetrics; use crate::opts::StorageOpts; + use crate::store::SealCurrentEpochOptions; const INITIAL_EPOCH: HummockEpoch = test_epoch(5); pub(crate) const TEST_TABLE_ID: TableId = TableId { table_id: 233 }; @@ -1313,6 +1564,16 @@ pub(crate) mod tests { )]) } + impl HummockUploader { + fn local_seal_epoch_for_test(&mut self, instance_id: LocalInstanceId, epoch: HummockEpoch) { + self.local_seal_epoch( + instance_id, + epoch.next_epoch(), + SealCurrentEpochOptions::for_test(), + ); + } + } + #[tokio::test] pub async fn test_uploading_task_future() { let uploader_context = test_uploader_context(dummy_success_upload_future); @@ -1389,36 +1650,15 @@ pub(crate) mod tests { let mut uploader = test_uploader(dummy_success_upload_future); let epoch1 = INITIAL_EPOCH.next_epoch(); let imm = gen_imm(epoch1).await; - + uploader.init_instance(TEST_LOCAL_INSTANCE_ID, TEST_TABLE_ID, epoch1); uploader.add_imm(TEST_LOCAL_INSTANCE_ID, imm.clone()); - assert_eq!(1, uploader.data().unsealed_data.len()); - assert_eq!( - epoch1 as HummockEpoch, - *uploader.data().unsealed_data.first_key_value().unwrap().0 - ); - assert_eq!( - 1, - uploader - .data() - .unsealed_data - .first_key_value() - .unwrap() - .1 - .imms - .len() - ); - uploader.seal_epoch(epoch1); - assert_eq!(epoch1, uploader.max_sealed_epoch); - assert!(uploader.data().unsealed_data.is_empty()); - assert_eq!(1, uploader.data().sealed_data.imm_count()); + uploader.local_seal_epoch_for_test(TEST_LOCAL_INSTANCE_ID, epoch1); uploader.start_sync_epoch_for_test(epoch1); assert_eq!(epoch1 as HummockEpoch, uploader.max_syncing_epoch); - assert_eq!(0, uploader.data().sealed_data.imm_count()); - assert!(uploader.data().sealed_data.spilled_data.is_empty()); assert_eq!(1, uploader.data().syncing_data.len()); let syncing_data = uploader.data().syncing_data.front().unwrap(); - assert_eq!(epoch1 as HummockEpoch, syncing_data.sync_epoch()); + assert_eq!(epoch1 as HummockEpoch, syncing_data.sync_epoch); assert!(syncing_data.uploaded.is_empty()); assert!(syncing_data.uploading_tasks.is_some()); @@ -1456,6 +1696,32 @@ pub(crate) mod tests { assert_eq!(epoch1, uploader.max_committed_epoch()); } + #[tokio::test] + async fn test_empty_uploader_sync() { + let mut uploader = test_uploader(dummy_success_upload_future); + let epoch1 = INITIAL_EPOCH.next_epoch(); + + uploader.start_sync_epoch_for_test(epoch1); + assert_eq!(epoch1, uploader.max_syncing_epoch); + + match uploader.next_event().await { + UploaderEvent::SyncFinish(finished_epoch, data) => { + assert_eq!(epoch1, finished_epoch); + assert!(data.uploaded_ssts.is_empty()); + assert!(data.newly_upload_ssts.is_empty()); + } + _ => unreachable!(), + }; + assert_eq!(epoch1, uploader.max_synced_epoch()); + let new_pinned_version = uploader + .context + .pinned_version + .new_pin_version(test_hummock_version(epoch1)); + uploader.update_pinned_version(new_pinned_version); + assert!(uploader.data().syncing_data.is_empty()); + assert_eq!(epoch1, uploader.max_committed_epoch()); + } + #[tokio::test] async fn test_uploader_empty_epoch() { let mut uploader = test_uploader(dummy_success_upload_future); @@ -1463,9 +1729,9 @@ pub(crate) mod tests { let epoch2 = epoch1.next_epoch(); let imm = gen_imm(epoch2).await; // epoch1 is empty while epoch2 is not. Going to seal empty epoch1. + uploader.init_instance(TEST_LOCAL_INSTANCE_ID, TEST_TABLE_ID, epoch1); + uploader.local_seal_epoch_for_test(TEST_LOCAL_INSTANCE_ID, epoch1); uploader.add_imm(TEST_LOCAL_INSTANCE_ID, imm); - uploader.seal_epoch(epoch1); - assert_eq!(epoch1, uploader.max_sealed_epoch); uploader.start_sync_epoch_for_test(epoch1); assert_eq!(epoch1, uploader.max_syncing_epoch); @@ -1495,12 +1761,7 @@ pub(crate) mod tests { assert!(poll_fn(|cx| data.poll_syncing_task(cx, &uploader.context)) .await .is_none()); - assert!(poll_fn(|cx| data.poll_sealed_spill_task(cx)) - .await - .is_none()); - assert!(poll_fn(|cx| data.poll_unsealed_spill_task(cx)) - .await - .is_none()); + assert!(poll_fn(|cx| data.poll_spill_task(cx)).await.is_none()); } #[tokio::test] @@ -1521,27 +1782,23 @@ pub(crate) mod tests { uploader.update_pinned_version(version1); assert_eq!(epoch1, uploader.max_synced_epoch); assert_eq!(epoch1, uploader.max_syncing_epoch); - assert_eq!(epoch1, uploader.max_sealed_epoch); + uploader.init_instance(TEST_LOCAL_INSTANCE_ID, TEST_TABLE_ID, epoch6); uploader.add_imm(TEST_LOCAL_INSTANCE_ID, gen_imm(epoch6).await); uploader.update_pinned_version(version2); assert_eq!(epoch2, uploader.max_synced_epoch); assert_eq!(epoch2, uploader.max_syncing_epoch); - assert_eq!(epoch2, uploader.max_sealed_epoch); - uploader.seal_epoch(epoch6); - assert_eq!(epoch6, uploader.max_sealed_epoch); + uploader.local_seal_epoch_for_test(TEST_LOCAL_INSTANCE_ID, epoch6); uploader.update_pinned_version(version3); assert_eq!(epoch3, uploader.max_synced_epoch); assert_eq!(epoch3, uploader.max_syncing_epoch); - assert_eq!(epoch6, uploader.max_sealed_epoch); uploader.start_sync_epoch_for_test(epoch6); assert_eq!(epoch6, uploader.max_syncing_epoch); uploader.update_pinned_version(version4); assert_eq!(epoch4, uploader.max_synced_epoch); assert_eq!(epoch6, uploader.max_syncing_epoch); - assert_eq!(epoch6, uploader.max_sealed_epoch); match uploader.next_event().await { UploaderEvent::SyncFinish(epoch, _) => { @@ -1552,7 +1809,6 @@ pub(crate) mod tests { uploader.update_pinned_version(version5); assert_eq!(epoch6, uploader.max_synced_epoch); assert_eq!(epoch6, uploader.max_syncing_epoch); - assert_eq!(epoch6, uploader.max_sealed_epoch); } fn prepare_uploader_order_test( @@ -1648,6 +1904,9 @@ pub(crate) mod tests { let instance_id1 = 1; let instance_id2 = 2; + uploader.init_instance(instance_id1, TEST_TABLE_ID, epoch1); + uploader.init_instance(instance_id2, TEST_TABLE_ID, epoch2); + // imm2 contains data in newer epoch, but added first let imm2 = gen_imm_with_limiter(epoch2, memory_limiter).await; uploader.add_imm(instance_id2, imm2.clone()); @@ -1700,18 +1959,19 @@ pub(crate) mod tests { let epoch1_sync_payload = HashMap::from_iter([(instance_id1, vec![imm1_4.clone()])]); let (await_start1_4, finish_tx1_4) = new_task_notifier(get_payload_imm_ids(&epoch1_sync_payload)); - uploader.seal_epoch(epoch1); + uploader.local_seal_epoch_for_test(instance_id1, epoch1); uploader.start_sync_epoch_for_test(epoch1); await_start1_4.await; + let epoch3 = epoch2.next_epoch(); - uploader.seal_epoch(epoch2); + uploader.local_seal_epoch_for_test(instance_id1, epoch2); + uploader.local_seal_epoch_for_test(instance_id2, epoch2); // current uploader state: // unsealed: empty // sealed: epoch2: uploaded sst([imm2]) // syncing: epoch1: uploading: [imm1_4], [imm1_3], uploaded: sst([imm1_2, imm1_1]) - let epoch3 = epoch2.next_epoch(); let imm3_1 = gen_imm_with_limiter(epoch3, memory_limiter).await; let epoch3_spill_payload1 = HashMap::from_iter([(instance_id1, vec![imm3_1.clone()])]); uploader.add_imm(instance_id1, imm3_1.clone()); @@ -1735,6 +1995,7 @@ pub(crate) mod tests { // syncing: epoch1: uploading: [imm1_4], [imm1_3], uploaded: sst([imm1_2, imm1_1]) let epoch4 = epoch3.next_epoch(); + uploader.local_seal_epoch_for_test(instance_id1, epoch3); let imm4 = gen_imm_with_limiter(epoch4, memory_limiter).await; uploader.add_imm(instance_id1, imm4.clone()); assert_uploader_pending(&mut uploader).await; @@ -1800,7 +2061,7 @@ pub(crate) mod tests { // synced: epoch1: sst([imm1_4]), sst([imm1_3]), sst([imm1_2, imm1_1]) // epoch2: sst([imm2]) - uploader.seal_epoch(epoch3); + uploader.local_seal_epoch_for_test(instance_id2, epoch3); if let UploaderEvent::DataSpilled(sst) = uploader.next_event().await { assert_eq!(&get_payload_imm_ids(&epoch3_spill_payload1), sst.imm_ids()); } else { @@ -1814,7 +2075,8 @@ pub(crate) mod tests { // synced: epoch1: sst([imm1_4]), sst([imm1_3]), sst([imm1_2, imm1_1]) // epoch2: sst([imm2]) - uploader.seal_epoch(epoch4); + uploader.local_seal_epoch_for_test(instance_id1, epoch4); + uploader.local_seal_epoch_for_test(instance_id2, epoch4); let epoch4_sync_payload = HashMap::from_iter([(instance_id1, vec![imm4, imm3_3])]); let (await_start4_with_3_3, finish_tx4_with_3_3) = new_task_notifier(get_payload_imm_ids(&epoch4_sync_payload)); @@ -1877,9 +2139,14 @@ pub(crate) mod tests { let epoch1 = INITIAL_EPOCH.next_epoch(); let epoch2 = epoch1.next_epoch(); + let instance_id1 = 1; + let instance_id2 = 2; let flush_threshold = buffer_tracker.flush_threshold(); let memory_limiter = buffer_tracker.get_memory_limiter().clone(); + uploader.init_instance(instance_id1, TEST_TABLE_ID, epoch1); + uploader.init_instance(instance_id2, TEST_TABLE_ID, epoch2); + // imm2 contains data in newer epoch, but added first let mut total_memory = 0; while total_memory < flush_threshold { @@ -1888,15 +2155,15 @@ pub(crate) mod tests { if total_memory > flush_threshold { break; } - uploader.add_imm(TEST_LOCAL_INSTANCE_ID, imm); + uploader.add_imm(instance_id2, imm); } let imm = gen_imm_with_limiter(epoch1, Some(memory_limiter.as_ref())).await; - uploader.add_imm(TEST_LOCAL_INSTANCE_ID, imm); + uploader.add_imm(instance_id1, imm); assert!(uploader.may_flush()); for _ in 0..10 { let imm = gen_imm_with_limiter(epoch1, Some(memory_limiter.as_ref())).await; - uploader.add_imm(TEST_LOCAL_INSTANCE_ID, imm); + uploader.add_imm(instance_id1, imm); assert!(!uploader.may_flush()); } } diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index e52b87c7d8aba..2d89fdd401fa2 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -583,12 +583,6 @@ impl StateStore for HummockStorage { MemOrdering::SeqCst, ); } - self.hummock_event_sender - .send(HummockEvent::SealEpoch { - epoch, - is_checkpoint, - }) - .expect("should send success"); StoreLocalStatistic::flush_all(); } diff --git a/src/storage/src/hummock/store/local_hummock_storage.rs b/src/storage/src/hummock/store/local_hummock_storage.rs index 2f0ad3437efce..a14f3b450adff 100644 --- a/src/storage/src/hummock/store/local_hummock_storage.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -513,6 +513,14 @@ impl LocalStateStore for LocalHummockStorage { "local state store of table id {:?} is init for more than once", self.table_id ); + if !self.is_replicated { + self.event_sender + .send(HummockEvent::InitEpoch { + instance_id: self.instance_id(), + init_epoch: options.epoch.curr, + }) + .expect("should succeed"); + } Ok(()) } @@ -544,14 +552,15 @@ impl LocalStateStore for LocalHummockStorage { }); } } - self.event_sender - .send(HummockEvent::LocalSealEpoch { - instance_id: self.instance_id(), - table_id: self.table_id, - epoch: prev_epoch, - opts, - }) - .expect("should be able to send") + if !self.is_replicated { + self.event_sender + .send(HummockEvent::LocalSealEpoch { + instance_id: self.instance_id(), + next_epoch, + opts, + }) + .expect("should be able to send"); + } } fn update_vnode_bitmap(&mut self, vnodes: Arc) -> Arc { diff --git a/src/stream/src/executor/actor.rs b/src/stream/src/executor/actor.rs index 249546031e229..1c73a3aeddad6 100644 --- a/src/stream/src/executor/actor.rs +++ b/src/stream/src/executor/actor.rs @@ -223,14 +223,15 @@ where ) .into())); - // Collect barriers to local barrier manager - self.barrier_manager.collect(id, &barrier); - // Then stop this actor if asked if barrier.is_stop(id) { - break Ok(()); + debug!(actor_id = id, epoch = ?barrier.epoch, "stop at barrier"); + break Ok(barrier); } + // Collect barriers to local barrier manager + self.barrier_manager.collect(id, &barrier); + // Tracing related work last_epoch = Some(barrier.epoch); span = barrier.tracing_context().attach(new_span(last_epoch)); @@ -238,7 +239,12 @@ where spawn_blocking_drop_stream(stream).await; - tracing::trace!(actor_id = id, "actor exit"); + let result = result.map(|stop_barrier| { + // Collect the stop barrier after the stream has been dropped to ensure that all resources + self.barrier_manager.collect(id, &stop_barrier); + }); + + tracing::debug!(actor_id = id, ok = result.is_ok(), "actor exit"); result } } diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 557dda4f535ef..7920e8dceee80 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -551,7 +551,8 @@ where // If not finished then we need to update state, otherwise no need. if let Message::Barrier(barrier) = &msg { if is_completely_finished { - // If already finished, no need to persist any state. + // If already finished, no need to persist any state. But we need to advance the epoch anyway + self.state_table.commit(barrier.epoch).await?; } else { // If snapshot was empty, we do not need to backfill, // but we still need to persist the finished state. @@ -595,6 +596,10 @@ where #[for_await] for msg in upstream { if let Some(msg) = mapping_message(msg?, &self.output_indices) { + if let Message::Barrier(barrier) = &msg { + // If already finished, no need persist any state, but we need to advance the epoch of the state table anyway. + self.state_table.commit(barrier.epoch).await?; + } yield msg; } } diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index bd130fd8f52a2..e368086a97737 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -498,7 +498,10 @@ where // If not finished then we need to update state, otherwise no need. if let Message::Barrier(barrier) = &msg { if is_finished { - // If already finished, no need persist any state. + // If already finished, no need persist any state, but we need to advance the epoch of the state table anyway. + if let Some(table) = &mut self.state_table { + table.commit(barrier.epoch).await?; + } } else { // If snapshot was empty, we do not need to backfill, // but we still need to persist the finished state. @@ -564,6 +567,13 @@ where #[for_await] for msg in upstream { if let Some(msg) = mapping_message(msg?, &self.output_indices) { + if let Message::Barrier(barrier) = &msg { + // If already finished, no need persist any state, but we need to advance the epoch of the state table anyway. + if let Some(table) = &mut self.state_table { + table.commit(barrier.epoch).await?; + } + } + yield msg; } } From 51fccb75d26751a9f385f4df07185cd22261c678 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Thu, 13 Jun 2024 10:11:45 +0800 Subject: [PATCH 27/37] feat(connector): add SQL Server sink (#17154) --- Cargo.lock | 155 ++++++ ci/docker-compose.yml | 11 + ci/scripts/e2e-sqlserver-sink-test.sh | 80 ++++ ci/workflows/main-cron.yml | 19 + ci/workflows/pull-request.yml | 15 + e2e_test/sink/sqlserver_sink.slt | 59 +++ src/connector/Cargo.toml | 1 + src/connector/src/sink/mod.rs | 14 + src/connector/src/sink/sqlserver.rs | 649 ++++++++++++++++++++++++++ src/connector/src/with_options.rs | 1 + src/connector/with_options_sink.yaml | 27 ++ 11 files changed, 1031 insertions(+) create mode 100755 ci/scripts/e2e-sqlserver-sink-test.sh create mode 100644 e2e_test/sink/sqlserver_sink.slt create mode 100644 src/connector/src/sink/sqlserver.rs diff --git a/Cargo.lock b/Cargo.lock index a3e602a653713..462b542829cee 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1097,6 +1097,19 @@ dependencies = [ "syn 2.0.57", ] +[[package]] +name = "asynchronous-codec" +version = "0.6.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4057f2c32adbb2fc158e22fb38433c8e9bbf76b75a4732c7c0cbaf695fb65568" +dependencies = [ + "bytes", + "futures-sink", + "futures-util", + "memchr", + "pin-project-lite", +] + [[package]] name = "atoi" version = "2.0.0" @@ -2654,6 +2667,12 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "connection-string" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "510ca239cf13b7f8d16a2b48f263de7b4f8c566f0af58d901031473c76afb1e3" + [[package]] name = "console" version = "0.15.7" @@ -4540,6 +4559,70 @@ version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a357d28ed41a50f9c765dbfe56cbc04a64e53e5fc58ba79fbc34c10ef3df831f" +[[package]] +name = "encoding" +version = "0.2.33" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b0d943856b990d12d3b55b359144ff341533e516d94098b1d3fc1ac666d36ec" +dependencies = [ + "encoding-index-japanese", + "encoding-index-korean", + "encoding-index-simpchinese", + "encoding-index-singlebyte", + "encoding-index-tradchinese", +] + +[[package]] +name = "encoding-index-japanese" +version = "1.20141219.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04e8b2ff42e9a05335dbf8b5c6f7567e5591d0d916ccef4e0b1710d32a0d0c91" +dependencies = [ + "encoding_index_tests", +] + +[[package]] +name = "encoding-index-korean" +version = "1.20141219.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4dc33fb8e6bcba213fe2f14275f0963fd16f0a02c878e3095ecfdf5bee529d81" +dependencies = [ + "encoding_index_tests", +] + +[[package]] +name = "encoding-index-simpchinese" +version = "1.20141219.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d87a7194909b9118fc707194baa434a4e3b0fb6a5a757c73c3adb07aa25031f7" +dependencies = [ + "encoding_index_tests", +] + +[[package]] +name = "encoding-index-singlebyte" +version = "1.20141219.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3351d5acffb224af9ca265f435b859c7c01537c0849754d3db3fdf2bfe2ae84a" +dependencies = [ + "encoding_index_tests", +] + +[[package]] +name = "encoding-index-tradchinese" +version = "1.20141219.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fd0e20d5688ce3cab59eb3ef3a2083a5c77bf496cb798dc6fcdb75f323890c18" +dependencies = [ + "encoding_index_tests", +] + +[[package]] +name = "encoding_index_tests" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a246d82be1c9d791c5dfde9a2bd045fc3cbba3fa2b11ad558f27d01712f00569" + [[package]] name = "encoding_rs" version = "0.8.33" @@ -8955,6 +9038,12 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "04bfa62906ce8d9badf8d1764501640ae7f0bcea3437a209315830e0f73564d1" +[[package]] +name = "pretty-hex" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c6fa0831dd7cc608c38a5e323422a0077678fa5744aa2be4ad91c4ece8eec8d5" + [[package]] name = "pretty-xmlish" version = "0.1.13" @@ -10653,6 +10742,7 @@ dependencies = [ "tempfile", "thiserror", "thiserror-ext", + "tiberius", "time", "tokio-postgres", "tokio-retry", @@ -12000,6 +12090,18 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "rustls" +version = "0.20.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b80e3dec595989ea8510028f30c408a4630db12c9cbb8de34203b89d6577e99" +dependencies = [ + "log", + "ring 0.16.20", + "sct", + "webpki", +] + [[package]] name = "rustls" version = "0.21.11" @@ -14303,6 +14405,37 @@ dependencies = [ "ordered-float 2.10.0", ] +[[package]] +name = "tiberius" +version = "0.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc6e2bf3e4b5be181a2a2ceff4b9b12e2684010d436a6958bd564fbc8094d44d" +dependencies = [ + "async-trait", + "asynchronous-codec", + "bigdecimal 0.3.1", + "byteorder", + "bytes", + "chrono", + "connection-string", + "encoding", + "enumflags2", + "futures-util", + "num-traits", + "once_cell", + "pin-project-lite", + "pretty-hex", + "rust_decimal", + "rustls-native-certs 0.6.3", + "rustls-pemfile 1.0.4", + "thiserror", + "time", + "tokio-rustls 0.23.4", + "tokio-util", + "tracing", + "uuid", +] + [[package]] name = "tikv-jemalloc-ctl" version = "0.5.4" @@ -14506,6 +14639,17 @@ dependencies = [ "rand", ] +[[package]] +name = "tokio-rustls" +version = "0.23.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" +dependencies = [ + "rustls 0.20.9", + "tokio", + "webpki", +] + [[package]] name = "tokio-rustls" version = "0.24.1" @@ -14568,6 +14712,7 @@ checksum = "1d68074620f57a0b21594d9735eb2e98ab38b17f80d3fcb189fca266771ca60d" dependencies = [ "bytes", "futures-core", + "futures-io", "futures-sink", "pin-project-lite", "tokio", @@ -15918,6 +16063,16 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "webpki" +version = "0.22.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed63aea5ce73d0ff405984102c42de94fc55a6b75765d621c65262469b3c9b53" +dependencies = [ + "ring 0.17.5", + "untrusted 0.9.0", +] + [[package]] name = "webpki-roots" version = "0.25.2" diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index 4786f85327d14..dd10bc8c98c1d 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -100,6 +100,7 @@ services: - doris-server - starrocks-fe-server - starrocks-be-server + - sqlserver-server volumes: - ..:/risingwave @@ -204,6 +205,16 @@ services: timeout: 5s retries: 30 + sqlserver-server: + container_name: sqlserver-server + image: mcr.microsoft.com/mssql/server:2022-latest + hostname: sqlserver-server + ports: + - 1433:1433 + environment: + ACCEPT_EULA: 'Y' + SA_PASSWORD: 'SomeTestOnly@SA' + starrocks-fe-server: container_name: starrocks-fe-server image: starrocks/fe-ubuntu:3.1.7 diff --git a/ci/scripts/e2e-sqlserver-sink-test.sh b/ci/scripts/e2e-sqlserver-sink-test.sh new file mode 100755 index 0000000000000..f1f62941375ce --- /dev/null +++ b/ci/scripts/e2e-sqlserver-sink-test.sh @@ -0,0 +1,80 @@ +#!/usr/bin/env bash + +# Exits as soon as any line fails. +set -euo pipefail + +source ci/scripts/common.sh + +while getopts 'p:' opt; do + case ${opt} in + p ) + profile=$OPTARG + ;; + \? ) + echo "Invalid Option: -$OPTARG" 1>&2 + exit 1 + ;; + : ) + echo "Invalid option: $OPTARG requires an argument" 1>&2 + ;; + esac +done +shift $((OPTIND -1)) + +download_and_prepare_rw "$profile" source + +echo "--- starting risingwave cluster" +risedev ci-start ci-sink-test +sleep 1 + +echo "--- create SQL Server table" +curl https://packages.microsoft.com/keys/microsoft.asc | sudo apt-key add - +curl https://packages.microsoft.com/config/ubuntu/20.04/prod.list | sudo tee /etc/apt/sources.list.d/msprod.list +apt-get update -y +ACCEPT_EULA=Y DEBIAN_FRONTEND=noninteractive apt-get install -y mssql-tools unixodbc-dev +export PATH="/opt/mssql-tools/bin/:$PATH" +sleep 2 + +sqlcmd -S sqlserver-server -U SA -P SomeTestOnly@SA -Q " +CREATE DATABASE SinkTest; +GO +USE SinkTest; +CREATE TABLE t_many_data_type ( + k1 int, k2 int, + c_boolean bit, + c_int16 smallint, + c_int32 int, + c_int64 bigint, + c_float32 float, + c_float64 float, + c_decimal decimal, + c_date date, + c_time time, + c_timestamp datetime2, + c_timestampz datetime2, + c_nvarchar nvarchar(1024), + c_varbinary varbinary(1024), +PRIMARY KEY (k1,k2)); +GO" +sleep 2 + +echo "--- testing sinks" +sqllogictest -p 4566 -d dev './e2e_test/sink/sqlserver_sink.slt' +sleep 1 +sqlcmd -S sqlserver-server -U SA -P SomeTestOnly@SA -h -1 -Q " +SELECT * FROM SinkTest.dbo.t_many_data_type; +GO" > ./query_result.txt + +mapfile -t actual < <(tr -s '[:space:]' '\n' < query_result.txt) +actual=("${actual[@]:1}") +expected=(0 0 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0 55 55 1 1.0 1.0 1 2022-04-08 18:20:49.0000000 2022-03-13 01:00:00.0000000 2022-03-13 01:00:00.0000000 Hello World! 0xDE00BEEF 1 2 0 66 66 1 1.0 1.0 1 2022-04-08 18:20:49.0000000 2022-03-13 01:00:00.0000000 2022-03-13 01:00:00.0000000 Hello World! 0xDE00BEEF 1 4 0 2 2 1 1.0 1.0 1 2022-04-08 18:20:49.0000000 2022-03-13 01:00:00.0000000 2022-03-13 01:00:00.0000000 Hello World! 0xDE00BEEF "(4" rows "affected)") + +if [[ ${#actual[@]} -eq ${#expected[@]} && ${actual[@]} == ${expected[@]} ]]; then + echo "SQL Server sink check passed" +else + cat ./query_result.txt + echo "The output is not as expected." +fi + +echo "--- Kill cluster" +risedev ci-kill diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 7a1054a0d481b..c9eaf5cf0c38d 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -868,6 +868,25 @@ steps: timeout_in_minutes: 10 retry: *auto-retry + - label: "end-to-end sqlserver sink test" + key: "e2e-sqlserver-sink-tests" + command: "ci/scripts/e2e-sqlserver-sink-test.sh -p ci-release" + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-sqlserver-sink-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-sqlserver-sink-tests?(,|$$)/ + depends_on: + - "build" + - "build-other" + plugins: + - docker-compose#v5.1.0: + run: sink-test-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 10 + retry: *auto-retry + - label: "end-to-end pulsar sink test" key: "e2e-pulsar-sink-tests" command: "ci/scripts/e2e-pulsar-sink-test.sh -p ci-release" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 385790e830232..20f0ef4128247 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -315,6 +315,21 @@ steps: timeout_in_minutes: 10 retry: *auto-retry + - label: "end-to-end sqlserver sink test" + if: build.pull_request.labels includes "ci/run-e2e-sqlserver-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-sqlserver-sink-tests?(,|$$)/ + command: "ci/scripts/e2e-sqlserver-sink-test.sh -p ci-dev" + depends_on: + - "build" + - "build-other" + plugins: + - docker-compose#v5.1.0: + run: sink-test-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 10 + retry: *auto-retry + - label: "end-to-end deltalake sink test" if: build.pull_request.labels includes "ci/run- e2e-deltalake-sink-rust-tests" || build.env("CI_STEPS") =~ /(^|,) e2e-deltalake-sink-rust-tests?(,|$$)/ command: "ci/scripts/e2e-deltalake-sink-rust-test.sh -p ci-dev" diff --git a/e2e_test/sink/sqlserver_sink.slt b/e2e_test/sink/sqlserver_sink.slt new file mode 100644 index 0000000000000..156b8b865ffc8 --- /dev/null +++ b/e2e_test/sink/sqlserver_sink.slt @@ -0,0 +1,59 @@ +statement ok +create table t_many_data_type_rw ( + k1 int, k2 int, + c_boolean bool, + c_int16 smallint, + c_int32 int, + c_int64 bigint, + c_float32 float, + c_float64 double, + c_decimal decimal, + c_date date, + c_time time, + c_timestamp timestamp, + c_timestampz timestamp, + c_nvarchar string, + c_varbinary bytea); + +statement ok +create sink s_many_data_type from t_many_data_type_rw with ( + connector = 'sqlserver', + type = 'upsert', + sqlserver.host = 'sqlserver-server', + sqlserver.port = 1433, + sqlserver.user = 'SA', + sqlserver.password = 'SomeTestOnly@SA', + sqlserver.database = 'SinkTest', + sqlserver.table = 't_many_data_type', + primary_key = 'k1,k2', +); + +statement ok +insert into t_many_data_type_rw values +(0,0,false,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL), +(1,1,false,1,1,1,1.0,1.0,1.0,date '2022-04-08',time '18:20:49','2022-03-13 01:00:00'::timestamp,'2022-03-13 01:00:00Z'::timestamptz,'Hello World!','\xDe00BeEf'), +(1,2,false,2,2,1,1.0,1.0,1.0,date '2022-04-08',time '18:20:49','2022-03-13 01:00:00'::timestamp,'2022-03-13 01:00:00Z'::timestamptz,'Hello World!','\xDe00BeEf'), +(1,3,false,2,2,1,1.0,1.0,1.0,date '2022-04-08',time '18:20:49','2022-03-13 01:00:00'::timestamp,'2022-03-13 01:00:00Z'::timestamptz,'Hello World!','\xDe00BeEf'), +(1,4,false,2,2,1,1.0,1.0,1.0,date '2022-04-08',time '18:20:49','2022-03-13 01:00:00'::timestamp,'2022-03-13 01:00:00Z'::timestamptz,'Hello World!','\xDe00BeEf'), +(1,1,false,2,2,1,1.0,1.0,1.0,date '2022-04-08',time '18:20:49','2022-03-13 01:00:00'::timestamp,'2022-03-13 01:00:00Z'::timestamptz,'Hello World!','\xDe00BeEf'); +flush; + +statement ok +delete from t_many_data_type_rw where k1=1 and k2=2; +delete from t_many_data_type_rw where k1=1 and k2=3; +flush; + +statement ok +insert into t_many_data_type_rw values +(1,1,false,55,55,1,1.0,1.0,1.0,date '2022-04-08',time '18:20:49','2022-03-13 01:00:00'::timestamp,'2022-03-13 01:00:00Z'::timestamptz,'Hello World!','\xDe00BeEf'), +(1,2,false,66,66,1,1.0,1.0,1.0,date '2022-04-08',time '18:20:49','2022-03-13 01:00:00'::timestamp,'2022-03-13 01:00:00Z'::timestamptz,'Hello World!','\xDe00BeEf'); +flush; + +statement ok +FLUSH; + +statement ok +DROP SINK s_many_data_type; + +statement ok +DROP TABLE t_many_data_type_rw; diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 1464fcc215b32..7cb6f23e5ec7e 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -136,6 +136,7 @@ strum_macros = "0.26" tempfile = "3" thiserror = "1" thiserror-ext = { workspace = true } +tiberius = { version = "0.12", default-features = false, features = ["chrono", "time", "tds73", "rust_decimal", "bigdecimal", "rustls"] } time = "0.3.30" tokio = { version = "0.2", package = "madsim-tokio", features = [ "rt", diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 6bfe20fa18c12..e5a5f6143e419 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -38,6 +38,7 @@ pub mod redis; pub mod remote; pub mod snowflake; pub mod snowflake_connector; +pub mod sqlserver; pub mod starrocks; pub mod test_sink; pub mod trivial; @@ -100,6 +101,7 @@ macro_rules! for_all_sinks { { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, { DynamoDb, $crate::sink::dynamodb::DynamoDbSink }, + { SqlServer, $crate::sink::sqlserver::SqlServerSink }, { Test, $crate::sink::test_sink::TestSink }, { Table, $crate::sink::trivial::TableSink } } @@ -577,6 +579,12 @@ pub enum SinkError { #[backtrace] anyhow::Error, ), + #[error("SQL Server error: {0}")] + SqlServer( + #[source] + #[backtrace] + anyhow::Error, + ), #[error(transparent)] Connector( #[from] @@ -614,3 +622,9 @@ impl From for SinkError { SinkError::Redis(value.to_report_string()) } } + +impl From for SinkError { + fn from(err: tiberius::error::Error) -> Self { + SinkError::SqlServer(anyhow!(err)) + } +} diff --git a/src/connector/src/sink/sqlserver.rs b/src/connector/src/sink/sqlserver.rs new file mode 100644 index 0000000000000..acdad3c47627e --- /dev/null +++ b/src/connector/src/sink/sqlserver.rs @@ -0,0 +1,649 @@ +// 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 std::collections::{BTreeMap, HashMap}; +use std::sync::Arc; + +use anyhow::anyhow; +use async_trait::async_trait; +use risingwave_common::array::{Op, RowRef, StreamChunk}; +use risingwave_common::buffer::Bitmap; +use risingwave_common::catalog::Schema; +use risingwave_common::row::{OwnedRow, Row}; +use risingwave_common::types::{DataType, Decimal}; +use serde_derive::Deserialize; +use serde_with::{serde_as, DisplayFromStr}; +use simd_json::prelude::ArrayTrait; +use tiberius::numeric::Numeric; +use tiberius::{AuthMethod, Client, ColumnData, Config, Query}; +use tokio::net::TcpStream; +use tokio_util::compat::TokioAsyncWriteCompatExt; +use with_options::WithOptions; + +use super::{SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; +use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriterParam}; + +pub const SQLSERVER_SINK: &str = "sqlserver"; + +fn default_max_batch_rows() -> usize { + 1024 +} + +#[serde_as] +#[derive(Clone, Debug, Deserialize, WithOptions)] +pub struct SqlServerConfig { + #[serde(rename = "sqlserver.host")] + pub host: String, + #[serde(rename = "sqlserver.port")] + #[serde_as(as = "DisplayFromStr")] + pub port: u16, + #[serde(rename = "sqlserver.user")] + pub user: String, + #[serde(rename = "sqlserver.password")] + pub password: String, + #[serde(rename = "sqlserver.database")] + pub database: String, + #[serde(rename = "sqlserver.table")] + pub table: String, + #[serde( + rename = "sqlserver.max_batch_rows", + default = "default_max_batch_rows" + )] + #[serde_as(as = "DisplayFromStr")] + pub max_batch_rows: usize, + pub r#type: String, // accept "append-only" or "upsert" +} + +impl SqlServerConfig { + pub fn from_btreemap(properties: BTreeMap) -> Result { + let config = + serde_json::from_value::(serde_json::to_value(properties).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) + } +} + +#[derive(Debug)] +pub struct SqlServerSink { + pub config: SqlServerConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, +} + +impl SqlServerSink { + pub fn new( + mut config: SqlServerConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + // Rewrite config because tiberius allows a maximum of 2100 params in one query request. + const TIBERIUS_PARAM_MAX: usize = 2000; + let params_per_op = schema.fields().len(); + let tiberius_max_batch_rows = if params_per_op == 0 { + config.max_batch_rows + } else { + ((TIBERIUS_PARAM_MAX as f64 / params_per_op as f64).floor()) as usize + }; + if tiberius_max_batch_rows == 0 { + return Err(SinkError::SqlServer(anyhow!(format!( + "too many column {}", + params_per_op + )))); + } + config.max_batch_rows = std::cmp::min(config.max_batch_rows, tiberius_max_batch_rows); + Ok(Self { + config, + schema, + pk_indices, + is_append_only, + }) + } +} + +impl TryFrom for SqlServerSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = SqlServerConfig::from_btreemap(param.properties)?; + SqlServerSink::new( + config, + schema, + param.downstream_pk, + param.sink_type.is_append_only(), + ) + } +} + +impl Sink for SqlServerSink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = LogSinkerOf; + + const SINK_NAME: &'static str = SQLSERVER_SINK; + + async fn validate(&self) -> Result<()> { + if !self.is_append_only && self.pk_indices.is_empty() { + return Err(SinkError::Config(anyhow!( + "Primary key not defined for upsert SQL Server sink (please define in `primary_key` field)"))); + } + + for f in self.schema.fields() { + check_data_type_compatibility(&f.data_type)?; + } + + // Query table metadata from SQL Server. + let mut sql_server_table_metadata = HashMap::new(); + let mut sql_client = SqlClient::new(&self.config).await?; + let query_table_metadata_error = || { + SinkError::SqlServer(anyhow!(format!( + "SQL Server table {} metadata error", + self.config.table + ))) + }; + static QUERY_TABLE_METADATA: &str = r#" +SELECT + col.name AS ColumnName, + pk.index_id AS PkIndex +FROM + sys.columns col +LEFT JOIN + sys.index_columns ic ON ic.object_id = col.object_id AND ic.column_id = col.column_id +LEFT JOIN + sys.indexes pk ON pk.object_id = col.object_id AND pk.index_id = ic.index_id AND pk.is_primary_key = 1 +WHERE + col.object_id = OBJECT_ID(@P1) +ORDER BY + col.column_id;"#; + let rows = sql_client + .client + .query(QUERY_TABLE_METADATA, &[&self.config.table]) + .await? + .into_results() + .await?; + for row in rows.into_iter().flatten() { + let mut iter = row.into_iter(); + let ColumnData::String(Some(col_name)) = + iter.next().ok_or_else(query_table_metadata_error)? + else { + return Err(query_table_metadata_error()); + }; + let ColumnData::I32(col_pk_index) = + iter.next().ok_or_else(query_table_metadata_error)? + else { + return Err(query_table_metadata_error()); + }; + sql_server_table_metadata.insert(col_name.into_owned(), col_pk_index.is_some()); + } + + // Validate Column name and Primary Key + for (idx, col) in self.schema.fields().iter().enumerate() { + let rw_is_pk = self.pk_indices.contains(&idx); + match sql_server_table_metadata.get(&col.name) { + None => { + return Err(SinkError::SqlServer(anyhow!(format!( + "column {} not found in the downstream SQL Server table {}", + col.name, self.config.table + )))); + } + Some(sql_server_is_pk) => { + if self.is_append_only { + continue; + } + if rw_is_pk && !*sql_server_is_pk { + return Err(SinkError::SqlServer(anyhow!(format!( + "column {} specified in primary_key mismatches with the downstream SQL Server table {} PK", + col.name, self.config.table, + )))); + } + if !rw_is_pk && *sql_server_is_pk { + return Err(SinkError::SqlServer(anyhow!(format!( + "column {} unspecified in primary_key mismatches with the downstream SQL Server table {} PK", + col.name, self.config.table, + )))); + } + } + } + } + + if !self.is_append_only { + let sql_server_pk_count = sql_server_table_metadata + .values() + .filter(|is_pk| **is_pk) + .count(); + if sql_server_pk_count != self.pk_indices.len() { + return Err(SinkError::SqlServer(anyhow!(format!( + "primary key does not match between RisingWave sink ({}) and SQL Server table {} ({})", + self.pk_indices.len(), + self.config.table, + sql_server_pk_count, + )))); + } + } + + Ok(()) + } + + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok(SqlServerSinkWriter::new( + self.config.clone(), + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + ) + .await? + .into_log_sinker(writer_param.sink_metrics)) + } +} + +enum SqlOp { + Insert(OwnedRow), + Merge(OwnedRow), + Delete(OwnedRow), +} + +pub struct SqlServerSinkWriter { + config: SqlServerConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + sql_client: SqlClient, + ops: Vec, +} + +impl SqlServerSinkWriter { + async fn new( + config: SqlServerConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + let sql_client = SqlClient::new(&config).await?; + let writer = Self { + config, + schema, + pk_indices, + is_append_only, + sql_client, + ops: vec![], + }; + Ok(writer) + } + + async fn delete_one(&mut self, row: RowRef<'_>) -> Result<()> { + if self.ops.len() + 1 >= self.config.max_batch_rows { + self.flush().await?; + } + self.ops.push(SqlOp::Delete(row.into_owned_row())); + Ok(()) + } + + async fn upsert_one(&mut self, row: RowRef<'_>) -> Result<()> { + if self.ops.len() + 1 >= self.config.max_batch_rows { + self.flush().await?; + } + self.ops.push(SqlOp::Merge(row.into_owned_row())); + Ok(()) + } + + async fn insert_one(&mut self, row: RowRef<'_>) -> Result<()> { + if self.ops.len() + 1 >= self.config.max_batch_rows { + self.flush().await?; + } + self.ops.push(SqlOp::Insert(row.into_owned_row())); + Ok(()) + } + + async fn flush(&mut self) -> Result<()> { + use std::fmt::Write; + if self.ops.is_empty() { + return Ok(()); + } + let mut query_str = String::new(); + let col_num = self.schema.fields.len(); + let mut next_param_id = 1; + let non_pk_col_indices = (0..col_num) + .filter(|idx| !self.pk_indices.contains(idx)) + .collect::>(); + let all_col_names = self + .schema + .fields + .iter() + .map(|f| format!("[{}]", f.name)) + .collect::>() + .join(","); + let all_source_col_names = self + .schema + .fields + .iter() + .map(|f| format!("[SOURCE].[{}]", f.name)) + .collect::>() + .join(","); + let pk_match = self + .pk_indices + .iter() + .map(|idx| { + format!( + "[SOURCE].[{}]=[TARGET].[{}]", + &self.schema[*idx].name, &self.schema[*idx].name + ) + }) + .collect::>() + .join(" AND "); + let param_placeholders = |param_id: &mut usize| { + let params = (*param_id..(*param_id + col_num)) + .map(|i| format!("@P{}", i)) + .collect::>() + .join(","); + *param_id += col_num; + params + }; + let set_all_source_col = non_pk_col_indices + .iter() + .map(|idx| { + format!( + "[{}]=[SOURCE].[{}]", + &self.schema[*idx].name, &self.schema[*idx].name + ) + }) + .collect::>() + .join(","); + // TODO: avoid repeating the SQL + for op in &self.ops { + match op { + SqlOp::Insert(_) => { + write!( + &mut query_str, + "INSERT INTO [{}] ({}) VALUES ({});", + self.config.table, + all_col_names, + param_placeholders(&mut next_param_id), + ) + .unwrap(); + } + SqlOp::Merge(_) => { + write!( + &mut query_str, + r#"MERGE [{}] AS [TARGET] + USING (VALUES ({})) AS [SOURCE] ({}) + ON {} + WHEN MATCHED THEN UPDATE SET {} + WHEN NOT MATCHED THEN INSERT ({}) VALUES ({});"#, + self.config.table, + param_placeholders(&mut next_param_id), + all_col_names, + pk_match, + set_all_source_col, + all_col_names, + all_source_col_names, + ) + .unwrap(); + } + SqlOp::Delete(_) => { + write!( + &mut query_str, + r#"DELETE FROM [{}] WHERE {};"#, + self.config.table, + self.pk_indices + .iter() + .map(|idx| { + let condition = + format!("[{}]=@P{}", self.schema[*idx].name, next_param_id); + next_param_id += 1; + condition + }) + .collect::>() + .join(" AND "), + ) + .unwrap(); + } + } + } + + let mut query = Query::new(query_str); + for op in self.ops.drain(..) { + match op { + SqlOp::Insert(row) => { + bind_params(&mut query, row, &self.schema, 0..col_num)?; + } + SqlOp::Merge(row) => { + bind_params(&mut query, row, &self.schema, 0..col_num)?; + } + SqlOp::Delete(row) => { + bind_params( + &mut query, + row, + &self.schema, + self.pk_indices.iter().copied(), + )?; + } + } + } + query.execute(&mut self.sql_client.client).await?; + Ok(()) + } +} + +#[async_trait] +impl SinkWriter for SqlServerSinkWriter { + async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { + Ok(()) + } + + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + for (op, row) in chunk.rows() { + match op { + Op::Insert => { + if self.is_append_only { + self.insert_one(row).await?; + } else { + self.upsert_one(row).await?; + } + } + Op::UpdateInsert => { + debug_assert!(!self.is_append_only); + self.upsert_one(row).await?; + } + Op::Delete => { + debug_assert!(!self.is_append_only); + self.delete_one(row).await?; + } + Op::UpdateDelete => {} + } + } + Ok(()) + } + + async fn barrier(&mut self, is_checkpoint: bool) -> Result { + if is_checkpoint { + self.flush().await?; + } + Ok(()) + } + + async fn abort(&mut self) -> Result<()> { + Ok(()) + } + + async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { + Ok(()) + } +} + +struct SqlClient { + client: Client>, +} + +impl SqlClient { + async fn new(msconfig: &SqlServerConfig) -> Result { + let mut config = Config::new(); + config.host(&msconfig.host); + config.port(msconfig.port); + config.authentication(AuthMethod::sql_server(&msconfig.user, &msconfig.password)); + config.database(&msconfig.database); + config.trust_cert(); + + let tcp = TcpStream::connect(config.get_addr()).await.unwrap(); + tcp.set_nodelay(true).unwrap(); + let client = Client::connect(config, tcp.compat_write()).await?; + Ok(Self { client }) + } +} + +fn bind_params( + query: &mut Query<'_>, + row: impl Row, + schema: &Schema, + col_indices: impl Iterator, +) -> Result<()> { + use risingwave_common::types::ScalarRefImpl; + for col_idx in col_indices { + match row.datum_at(col_idx) { + Some(data_ref) => match data_ref { + ScalarRefImpl::Int16(v) => query.bind(v), + ScalarRefImpl::Int32(v) => query.bind(v), + ScalarRefImpl::Int64(v) => query.bind(v), + ScalarRefImpl::Float32(v) => query.bind(v.into_inner()), + ScalarRefImpl::Float64(v) => query.bind(v.into_inner()), + ScalarRefImpl::Utf8(v) => query.bind(v.to_owned()), + ScalarRefImpl::Bool(v) => query.bind(v), + ScalarRefImpl::Decimal(v) => match v { + Decimal::Normalized(d) => { + query.bind(decimal_to_sql(&d)); + } + Decimal::NaN | Decimal::PositiveInf | Decimal::NegativeInf => { + tracing::warn!( + "Inf, -Inf, Nan in RisingWave decimal is converted into SQL Server null!" + ); + query.bind(None as Option); + } + }, + ScalarRefImpl::Date(v) => query.bind(v.0), + ScalarRefImpl::Timestamp(v) => query.bind(v.0), + ScalarRefImpl::Timestamptz(v) => query.bind(v.timestamp_micros()), + ScalarRefImpl::Time(v) => query.bind(v.0), + ScalarRefImpl::Bytea(v) => query.bind(v.to_vec()), + ScalarRefImpl::Interval(_) => return Err(data_type_not_supported("Interval")), + ScalarRefImpl::Jsonb(_) => return Err(data_type_not_supported("Jsonb")), + ScalarRefImpl::Struct(_) => return Err(data_type_not_supported("Struct")), + ScalarRefImpl::List(_) => return Err(data_type_not_supported("List")), + ScalarRefImpl::Int256(_) => return Err(data_type_not_supported("Int256")), + ScalarRefImpl::Serial(_) => return Err(data_type_not_supported("Serial")), + }, + None => match schema[col_idx].data_type { + DataType::Boolean => { + query.bind(None as Option); + } + DataType::Int16 => { + query.bind(None as Option); + } + DataType::Int32 => { + query.bind(None as Option); + } + DataType::Int64 => { + query.bind(None as Option); + } + DataType::Float32 => { + query.bind(None as Option); + } + DataType::Float64 => { + query.bind(None as Option); + } + DataType::Decimal => { + query.bind(None as Option); + } + DataType::Date => { + query.bind(None as Option); + } + DataType::Time => { + query.bind(None as Option); + } + DataType::Timestamp => { + query.bind(None as Option); + } + DataType::Timestamptz => { + query.bind(None as Option); + } + DataType::Varchar => { + query.bind(None as Option); + } + DataType::Bytea => { + query.bind(None as Option>); + } + DataType::Interval => return Err(data_type_not_supported("Interval")), + DataType::Struct(_) => return Err(data_type_not_supported("Struct")), + DataType::List(_) => return Err(data_type_not_supported("List")), + DataType::Jsonb => return Err(data_type_not_supported("Jsonb")), + DataType::Serial => return Err(data_type_not_supported("Serial")), + DataType::Int256 => return Err(data_type_not_supported("Int256")), + }, + }; + } + Ok(()) +} + +fn data_type_not_supported(data_type_name: &str) -> SinkError { + SinkError::SqlServer(anyhow!(format!( + "{data_type_name} is not supported in SQL Server" + ))) +} + +fn check_data_type_compatibility(data_type: &DataType) -> Result<()> { + match data_type { + DataType::Boolean + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Decimal + | DataType::Date + | DataType::Varchar + | DataType::Time + | DataType::Timestamp + | DataType::Timestamptz + | DataType::Bytea => Ok(()), + DataType::Interval => Err(data_type_not_supported("Interval")), + DataType::Struct(_) => Err(data_type_not_supported("Struct")), + DataType::List(_) => Err(data_type_not_supported("List")), + DataType::Jsonb => Err(data_type_not_supported("Jsonb")), + DataType::Serial => Err(data_type_not_supported("Serial")), + DataType::Int256 => Err(data_type_not_supported("Int256")), + } +} + +/// The implementation is copied from tiberius crate. +fn decimal_to_sql(decimal: &rust_decimal::Decimal) -> Numeric { + let unpacked = decimal.unpack(); + + let mut value = (((unpacked.hi as u128) << 64) + + ((unpacked.mid as u128) << 32) + + unpacked.lo as u128) as i128; + + if decimal.is_sign_negative() { + value = -value; + } + + Numeric::new_with_scale(value, decimal.scale() as u8) +} diff --git a/src/connector/src/with_options.rs b/src/connector/src/with_options.rs index 7e0eb4ce71604..3207a7bbbde2f 100644 --- a/src/connector/src/with_options.rs +++ b/src/connector/src/with_options.rs @@ -53,6 +53,7 @@ impl WithOptions for BTreeMap {} impl WithOptions for String {} impl WithOptions for bool {} impl WithOptions for usize {} +impl WithOptions for u16 {} impl WithOptions for u32 {} impl WithOptions for u64 {} impl WithOptions for i32 {} diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 54bde5a8325e2..731bb900335ee 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -751,6 +751,33 @@ SnowflakeConfig: field_type: String comments: The s3 region, e.g., us-east-2 required: true +SqlServerConfig: + fields: + - name: sqlserver.host + field_type: String + required: true + - name: sqlserver.port + field_type: u16 + required: true + - name: sqlserver.user + field_type: String + required: true + - name: sqlserver.password + field_type: String + required: true + - name: sqlserver.database + field_type: String + required: true + - name: sqlserver.table + field_type: String + required: true + - name: sqlserver.max_batch_rows + field_type: usize + required: false + default: '1024' + - name: r#type + field_type: String + required: true StarrocksConfig: fields: - name: starrocks.host From 3e1ee0d78d4a97d6c23830c198d242233a53264b Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Thu, 13 Jun 2024 10:11:55 +0800 Subject: [PATCH 28/37] fix(meta): include secrets in metadata backup (#17225) --- ci/workflows/pull-request.yml | 18 ++ proto/backup_service.proto | 1 + src/meta/model_v2/src/secret.rs | 3 +- .../meta_snapshot_builder_v2.rs | 168 ++----------- .../src/backup_restore/restore_impl/v2.rs | 46 ++-- src/storage/backup/src/lib.rs | 4 + src/storage/backup/src/meta_snapshot_v2.rs | 220 ++++++++---------- 7 files changed, 164 insertions(+), 296 deletions(-) diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 20f0ef4128247..cc5e670fe078a 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -121,6 +121,24 @@ steps: timeout_in_minutes: 8 retry: *auto-retry + - label: "meta backup test" + key: "e2e-meta-backup-test" + command: "ci/scripts/run-meta-backup-test.sh -p ci-dev -m ci-3streaming-2serving-3fe" + if: | + build.pull_request.labels includes "ci/run-e2e-meta-backup-test" + depends_on: + - "build" + - "build-other" + - "docslt" + plugins: + - docker-compose#v5.1.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 45 + retry: *auto-retry + - label: "end-to-end test (parallel)" command: "ci/scripts/e2e-test-parallel.sh -p ci-dev" if: | diff --git a/proto/backup_service.proto b/proto/backup_service.proto index 75420149042d9..48fe46ed7eac2 100644 --- a/proto/backup_service.proto +++ b/proto/backup_service.proto @@ -49,6 +49,7 @@ message MetaSnapshotMetadata { uint64 safe_epoch = 4; optional uint32 format_version = 5; optional string remarks = 6; + optional string rw_version = 7; } service BackupService { diff --git a/src/meta/model_v2/src/secret.rs b/src/meta/model_v2/src/secret.rs index af3590dd0de58..0d122267bb4bc 100644 --- a/src/meta/model_v2/src/secret.rs +++ b/src/meta/model_v2/src/secret.rs @@ -15,8 +15,9 @@ use risingwave_pb::catalog::PbSecret; use sea_orm::entity::prelude::*; use sea_orm::ActiveValue::Set; +use serde::{Deserialize, Serialize}; -#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize)] #[sea_orm(table_name = "secret")] pub struct Model { #[sea_orm(primary_key, auto_increment = false)] diff --git a/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs b/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs index c2bafcd071c68..4a81ae9d6ad08 100644 --- a/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs +++ b/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs @@ -31,6 +31,25 @@ fn map_db_err(e: DbErr) -> BackupError { BackupError::MetaStorage(e.into()) } +macro_rules! define_set_metadata { + ($( {$name:ident, $mod_path:ident::$mod_name:ident} ),*) => { + pub async fn set_metadata( + metadata: &mut MetadataV2, + txn: &sea_orm::DatabaseTransaction, + ) -> BackupResult<()> { + $( + metadata.$name = $mod_path::$mod_name::Entity::find() + .all(txn) + .await + .map_err(map_db_err)?; + )* + Ok(()) + } + }; +} + +risingwave_backup::for_all_metadata_models_v2!(define_set_metadata); + pub struct MetaSnapshotV2Builder { snapshot: MetaSnapshotV2, meta_store: SqlMetaStore, @@ -91,151 +110,14 @@ impl MetaSnapshotV2Builder { } redo_state }; - let version_stats = model_v2::prelude::HummockVersionStats::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let compaction_configs = model_v2::prelude::CompactionConfig::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let actors = model_v2::prelude::Actor::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let clusters = model_v2::prelude::Cluster::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let actor_dispatchers = model_v2::prelude::ActorDispatcher::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let catalog_versions = model_v2::prelude::CatalogVersion::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let connections = model_v2::prelude::Connection::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let databases = model_v2::prelude::Database::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let fragments = model_v2::prelude::Fragment::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let functions = model_v2::prelude::Function::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let indexes = model_v2::prelude::Index::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let objects = model_v2::prelude::Object::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let object_dependencies = model_v2::prelude::ObjectDependency::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let schemas = model_v2::prelude::Schema::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let sinks = model_v2::prelude::Sink::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let sources = model_v2::prelude::Source::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let streaming_jobs = model_v2::prelude::StreamingJob::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let subscriptions = model_v2::prelude::Subscription::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let system_parameters = model_v2::prelude::SystemParameter::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let tables = model_v2::prelude::Table::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let users = model_v2::prelude::User::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let user_privileges = model_v2::prelude::UserPrivilege::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let views = model_v2::prelude::View::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let workers = model_v2::prelude::Worker::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let worker_properties = model_v2::prelude::WorkerProperty::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let hummock_sequences = model_v2::prelude::HummockSequence::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let seaql_migrations = model_v2::serde_seaql_migration::Entity::find() - .all(&txn) - .await - .map_err(map_db_err)?; - let session_parameters = model_v2::prelude::SessionParameter::find() - .all(&txn) - .await - .map_err(map_db_err)?; - - txn.commit().await.map_err(map_db_err)?; - self.snapshot.metadata = MetadataV2 { - seaql_migrations, + let mut metadata = MetadataV2 { hummock_version, - version_stats, - compaction_configs, - actors, - clusters, - actor_dispatchers, - catalog_versions, - connections, - databases, - fragments, - functions, - indexes, - objects, - object_dependencies, - schemas, - sinks, - sources, - streaming_jobs, - subscriptions, - system_parameters, - tables, - users, - user_privileges, - views, - workers, - worker_properties, - hummock_sequences, - session_parameters, + ..Default::default() }; + set_metadata(&mut metadata, &txn).await?; + + txn.commit().await.map_err(map_db_err)?; + self.snapshot.metadata = metadata; Ok(()) } diff --git a/src/meta/src/backup_restore/restore_impl/v2.rs b/src/meta/src/backup_restore/restore_impl/v2.rs index cccb5c9b641d7..13492c56316a2 100644 --- a/src/meta/src/backup_restore/restore_impl/v2.rs +++ b/src/meta/src/backup_restore/restore_impl/v2.rs @@ -93,40 +93,28 @@ impl WriterModelV2ToMetaStoreV2 { } } +macro_rules! define_write_model_v2_to_meta_store_v2 { + ($( {$name:ident, $mod_path:ident::$mod_name:ident} ),*) => { + async fn write_model_v2_to_meta_store_v2( + metadata: &risingwave_backup::meta_snapshot_v2::MetadataV2, + db: &sea_orm::DatabaseConnection, + ) -> BackupResult<()> { + $( + insert_models(metadata.$name.clone(), db).await?; + )* + Ok(()) + } + }; +} + +risingwave_backup::for_all_metadata_models_v2!(define_write_model_v2_to_meta_store_v2); + #[async_trait::async_trait] impl Writer for WriterModelV2ToMetaStoreV2 { async fn write(&self, target_snapshot: MetaSnapshot) -> BackupResult<()> { let metadata = target_snapshot.metadata; let db = &self.meta_store.conn; - insert_models(metadata.seaql_migrations.clone(), db).await?; - insert_models(metadata.clusters.clone(), db).await?; - insert_models(metadata.version_stats.clone(), db).await?; - insert_models(metadata.compaction_configs.clone(), db).await?; - insert_models(metadata.hummock_sequences.clone(), db).await?; - insert_models(metadata.workers.clone(), db).await?; - insert_models(metadata.worker_properties.clone(), db).await?; - insert_models(metadata.users.clone(), db).await?; - insert_models(metadata.user_privileges.clone(), db).await?; - insert_models(metadata.objects.clone(), db).await?; - insert_models(metadata.object_dependencies.clone(), db).await?; - insert_models(metadata.databases.clone(), db).await?; - insert_models(metadata.schemas.clone(), db).await?; - insert_models(metadata.streaming_jobs.clone(), db).await?; - insert_models(metadata.fragments.clone(), db).await?; - insert_models(metadata.actors.clone(), db).await?; - insert_models(metadata.actor_dispatchers.clone(), db).await?; - insert_models(metadata.connections.clone(), db).await?; - insert_models(metadata.sources.clone(), db).await?; - insert_models(metadata.tables.clone(), db).await?; - insert_models(metadata.sinks.clone(), db).await?; - insert_models(metadata.views.clone(), db).await?; - insert_models(metadata.indexes.clone(), db).await?; - insert_models(metadata.functions.clone(), db).await?; - insert_models(metadata.system_parameters.clone(), db).await?; - insert_models(metadata.catalog_versions.clone(), db).await?; - insert_models(metadata.subscriptions.clone(), db).await?; - insert_models(metadata.session_parameters.clone(), db).await?; - + write_model_v2_to_meta_store_v2(&metadata, db).await?; // update_auto_inc must be called last. update_auto_inc(&metadata, db).await?; Ok(()) diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index 56c3ed551a282..ed8dccf8d1e49 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -36,6 +36,7 @@ use std::collections::HashSet; use std::hash::Hasher; use itertools::Itertools; +use risingwave_common::RW_VERSION; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{HummockSstableObjectId, HummockVersionId}; use risingwave_pb::backup_service::{PbMetaSnapshotManifest, PbMetaSnapshotMetadata}; @@ -57,6 +58,7 @@ pub struct MetaSnapshotMetadata { #[serde(default)] pub format_version: u32, pub remarks: Option, + pub rw_version: Option, } impl MetaSnapshotMetadata { @@ -74,6 +76,7 @@ impl MetaSnapshotMetadata { safe_epoch: v.visible_table_safe_epoch(), format_version, remarks, + rw_version: Some(RW_VERSION.to_owned()), } } } @@ -112,6 +115,7 @@ impl From<&MetaSnapshotMetadata> for PbMetaSnapshotMetadata { safe_epoch: m.safe_epoch, format_version: Some(m.format_version), remarks: m.remarks.clone(), + rw_version: m.rw_version.clone(), } } } diff --git a/src/storage/backup/src/meta_snapshot_v2.rs b/src/storage/backup/src/meta_snapshot_v2.rs index feb5f12540d03..bec07a80cf19d 100644 --- a/src/storage/backup/src/meta_snapshot_v2.rs +++ b/src/storage/backup/src/meta_snapshot_v2.rs @@ -16,7 +16,6 @@ use std::fmt::{Display, Formatter}; use bytes::{Buf, BufMut}; use risingwave_hummock_sdk::version::HummockVersion; -use risingwave_meta_model_v2 as model_v2; use serde::{Deserialize, Serialize}; use crate::meta_snapshot::{MetaSnapshot, Metadata}; @@ -29,39 +28,100 @@ impl From for BackupError { } } -#[derive(Default)] -pub struct MetadataV2 { - pub seaql_migrations: Vec, - pub hummock_version: HummockVersion, - pub version_stats: Vec, - pub compaction_configs: Vec, - pub actors: Vec, - pub clusters: Vec, - pub actor_dispatchers: Vec, - pub catalog_versions: Vec, - pub connections: Vec, - pub databases: Vec, - pub fragments: Vec, - pub functions: Vec, - pub indexes: Vec, - pub objects: Vec, - pub object_dependencies: Vec, - pub schemas: Vec, - pub sinks: Vec, - pub sources: Vec, - pub streaming_jobs: Vec, - pub subscriptions: Vec, - pub system_parameters: Vec, - pub tables: Vec, - pub users: Vec, - pub user_privileges: Vec, - pub views: Vec, - pub workers: Vec, - pub worker_properties: Vec, - pub hummock_sequences: Vec, - pub session_parameters: Vec, +/// Add new item in the end. Do not change the order. +#[macro_export] +macro_rules! for_all_metadata_models_v2 { + ($macro:ident) => { + $macro! { + {seaql_migrations, risingwave_meta_model_v2::serde_seaql_migration}, + {version_stats, risingwave_meta_model_v2::hummock_version_stats}, + {compaction_configs, risingwave_meta_model_v2::compaction_config}, + {actors, risingwave_meta_model_v2::actor}, + {clusters, risingwave_meta_model_v2::cluster}, + {actor_dispatchers, risingwave_meta_model_v2::actor_dispatcher}, + {catalog_versions, risingwave_meta_model_v2::catalog_version}, + {connections, risingwave_meta_model_v2::connection}, + {databases, risingwave_meta_model_v2::database}, + {fragments, risingwave_meta_model_v2::fragment}, + {functions, risingwave_meta_model_v2::function}, + {indexes, risingwave_meta_model_v2::index}, + {objects, risingwave_meta_model_v2::object}, + {object_dependencies, risingwave_meta_model_v2::object_dependency}, + {schemas, risingwave_meta_model_v2::schema}, + {sinks, risingwave_meta_model_v2::sink}, + {sources, risingwave_meta_model_v2::source}, + {streaming_jobs, risingwave_meta_model_v2::streaming_job}, + {subscriptions, risingwave_meta_model_v2::subscription}, + {system_parameters, risingwave_meta_model_v2::system_parameter}, + {tables, risingwave_meta_model_v2::table}, + {users, risingwave_meta_model_v2::user}, + {user_privileges, risingwave_meta_model_v2::user_privilege}, + {views, risingwave_meta_model_v2::view}, + {workers, risingwave_meta_model_v2::worker}, + {worker_properties, risingwave_meta_model_v2::worker_property}, + {hummock_sequences, risingwave_meta_model_v2::hummock_sequence}, + {session_parameters, risingwave_meta_model_v2::session_parameter}, + {secrets, risingwave_meta_model_v2::secret} + } + }; } +macro_rules! define_metadata_v2 { + ($({ $name:ident, $mod_path:ident::$mod_name:ident }),*) => { + #[derive(Default)] + pub struct MetadataV2 { + pub hummock_version: HummockVersion, + $( + pub $name: Vec<$mod_path::$mod_name::Model>, + )* + } + }; +} + +for_all_metadata_models_v2!(define_metadata_v2); + +macro_rules! define_encode_metadata { + ($( {$name:ident, $mod_path:ident::$mod_name:ident} ),*) => { + fn encode_metadata( + metadata: &MetadataV2, + buf: &mut Vec, + ) -> BackupResult<()> { + let mut _idx = 0; + $( + if _idx == 1 { + put_1(buf, &metadata.hummock_version.to_protobuf())?; + } + put_n(buf, &metadata.$name)?; + _idx += 1; + )* + Ok(()) + } + }; +} + +for_all_metadata_models_v2!(define_encode_metadata); + +macro_rules! define_decode_metadata { + ($( {$name:ident, $mod_path:ident::$mod_name:ident} ),*) => { + fn decode_metadata( + metadata: &mut MetadataV2, + mut buf: &[u8], + ) -> BackupResult<()> { + let mut _idx = 0; + $( + if _idx == 1 { + metadata.hummock_version = HummockVersion::from_persisted_protobuf(&get_1(&mut buf)?); + } + metadata.$name = get_n(&mut buf)?; + _idx += 1; + )* + Ok(()) + } + }; +} + +for_all_metadata_models_v2!(define_decode_metadata); + impl Display for MetadataV2 { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { writeln!(f, "clusters: {:#?}", self.clusters)?; @@ -77,102 +137,16 @@ impl Display for MetadataV2 { impl Metadata for MetadataV2 { fn encode_to(&self, buf: &mut Vec) -> BackupResult<()> { - put_n(buf, &self.seaql_migrations)?; - put_1(buf, &self.hummock_version.to_protobuf())?; - put_n(buf, &self.version_stats)?; - put_n(buf, &self.compaction_configs)?; - put_n(buf, &self.actors)?; - put_n(buf, &self.clusters)?; - put_n(buf, &self.actor_dispatchers)?; - put_n(buf, &self.catalog_versions)?; - put_n(buf, &self.connections)?; - put_n(buf, &self.databases)?; - put_n(buf, &self.fragments)?; - put_n(buf, &self.functions)?; - put_n(buf, &self.indexes)?; - put_n(buf, &self.objects)?; - put_n(buf, &self.object_dependencies)?; - put_n(buf, &self.schemas)?; - put_n(buf, &self.sinks)?; - put_n(buf, &self.sources)?; - put_n(buf, &self.streaming_jobs)?; - put_n(buf, &self.subscriptions)?; - put_n(buf, &self.system_parameters)?; - put_n(buf, &self.tables)?; - put_n(buf, &self.users)?; - put_n(buf, &self.user_privileges)?; - put_n(buf, &self.views)?; - put_n(buf, &self.workers)?; - put_n(buf, &self.worker_properties)?; - put_n(buf, &self.hummock_sequences)?; - put_n(buf, &self.session_parameters)?; - Ok(()) + encode_metadata(self, buf) } - fn decode(mut buf: &[u8]) -> BackupResult + fn decode(buf: &[u8]) -> BackupResult where Self: Sized, { - let seaql_migrations = get_n(&mut buf)?; - let pb_hummock_version = get_1(&mut buf)?; - let version_stats = get_n(&mut buf)?; - let compaction_configs = get_n(&mut buf)?; - let actors = get_n(&mut buf)?; - let clusters = get_n(&mut buf)?; - let actor_dispatchers = get_n(&mut buf)?; - let catalog_versions = get_n(&mut buf)?; - let connections = get_n(&mut buf)?; - let databases = get_n(&mut buf)?; - let fragments = get_n(&mut buf)?; - let functions = get_n(&mut buf)?; - let indexes = get_n(&mut buf)?; - let objects = get_n(&mut buf)?; - let object_dependencies = get_n(&mut buf)?; - let schemas = get_n(&mut buf)?; - let sinks = get_n(&mut buf)?; - let sources = get_n(&mut buf)?; - let streaming_jobs = get_n(&mut buf)?; - let subscriptions = get_n(&mut buf)?; - let system_parameters = get_n(&mut buf)?; - let tables = get_n(&mut buf)?; - let users = get_n(&mut buf)?; - let user_privileges = get_n(&mut buf)?; - let views = get_n(&mut buf)?; - let workers = get_n(&mut buf)?; - let worker_properties = get_n(&mut buf)?; - let hummock_sequences = get_n(&mut buf)?; - let session_parameters = get_n(&mut buf)?; - Ok(Self { - seaql_migrations, - hummock_version: HummockVersion::from_persisted_protobuf(&pb_hummock_version), - version_stats, - compaction_configs, - actors, - clusters, - actor_dispatchers, - catalog_versions, - connections, - databases, - fragments, - functions, - indexes, - objects, - object_dependencies, - schemas, - sinks, - sources, - streaming_jobs, - subscriptions, - system_parameters, - tables, - users, - user_privileges, - views, - workers, - worker_properties, - hummock_sequences, - session_parameters, - }) + let mut metadata = Self::default(); + decode_metadata(&mut metadata, buf)?; + Ok(metadata) } fn hummock_version_ref(&self) -> &HummockVersion { From 9abdbc9329f7108213264fc247906bfc07b39541 Mon Sep 17 00:00:00 2001 From: Patrick Lewis Date: Wed, 12 Jun 2024 22:16:04 -0400 Subject: [PATCH 29/37] chore(ci): remove obsolete version property from docker compose configs (#16566) --- ci/docker-compose.yml | 1 - docker/docker-compose-distributed.yml | 1 - docker/docker-compose-with-azblob.yml | 1 - docker/docker-compose-with-gcs.yml | 1 - docker/docker-compose-with-hdfs.yml | 1 - docker/docker-compose-with-local-fs.yml | 1 - docker/docker-compose-with-obs.yml | 1 - docker/docker-compose-with-oss.yml | 1 - docker/docker-compose-with-s3.yml | 1 - docker/docker-compose-with-sqlite.yml | 1 - docker/docker-compose.yml | 1 - integration_tests/ad-click/docker-compose.yml | 1 - integration_tests/ad-ctr/docker-compose.yml | 1 - integration_tests/big-query-sink/docker-compose.yml | 1 - .../cassandra-and-scylladb-sink/docker-compose.yml | 1 - integration_tests/cdn-metrics/docker-compose.yml | 1 - integration_tests/citus-cdc/docker-compose.yml | 1 - integration_tests/clickhouse-sink/docker-compose.yml | 1 - integration_tests/clickstream/docker-compose.yml | 1 - integration_tests/client-library/docker-compose.yml | 1 - integration_tests/cockroach-sink/docker-compose.yml | 1 - integration_tests/debezium-mysql/docker-compose.yml | 1 - integration_tests/debezium-postgres/docker-compose.yml | 1 - integration_tests/debezium-sqlserver/docker-compose.yml | 1 - integration_tests/deltalake-sink/docker-compose.yml | 1 - integration_tests/doris-sink/docker-compose.yml | 1 - integration_tests/elasticsearch-sink/docker-compose.yml | 1 - integration_tests/feature-store/docker-compose.yml | 1 - integration_tests/http-sink/docker-compose.yml | 1 - integration_tests/iceberg-sink/docker-compose.yml | 1 - integration_tests/kafka-cdc-sink/docker-compose.yml | 1 - integration_tests/kafka-cdc/docker-compose.yml | 1 - integration_tests/kinesis-s3-source/docker-compose.yml | 1 - integration_tests/livestream/docker-compose.yml | 1 - integration_tests/mindsdb/docker-compose.yml | 1 - integration_tests/mongodb-cdc/docker-compose.yaml | 1 - integration_tests/mongodb/docker-compose.yaml | 1 - integration_tests/mqtt/docker-compose.yml | 1 - integration_tests/mysql-cdc/docker-compose.yml | 1 - integration_tests/mysql-sink/docker-compose.yml | 1 - integration_tests/nats/docker-compose.yml | 1 - integration_tests/pinot-sink/docker-compose.yml | 1 - integration_tests/postgres-cdc/docker-compose.yml | 1 - integration_tests/postgres-sink/docker-compose.yml | 1 - integration_tests/presto-trino/docker-compose.yml | 1 - integration_tests/prometheus/docker-compose.yml | 1 - integration_tests/redis-sink/docker-compose.yml | 1 - integration_tests/schema-registry/docker-compose.yml | 1 - integration_tests/starrocks-sink/docker-compose.yml | 1 - integration_tests/superset/docker-compose.yml | 1 - integration_tests/twitter-pulsar/docker-compose.yml | 1 - integration_tests/twitter/docker-compose.yml | 1 - integration_tests/upsert-avro/docker-compose.yml | 3 +-- integration_tests/vector/docker-compose.yml | 1 - src/risedevtool/src/bin/risedev-compose.rs | 2 -- src/risedevtool/src/compose.rs | 1 - 56 files changed, 1 insertion(+), 58 deletions(-) diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index dd10bc8c98c1d..b2a885a4ba2e6 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -1,4 +1,3 @@ -version: "3.9" services: db: image: postgres:15-alpine diff --git a/docker/docker-compose-distributed.yml b/docker/docker-compose-distributed.yml index f94437f089197..80ce64cb90531 100644 --- a/docker/docker-compose-distributed.yml +++ b/docker/docker-compose-distributed.yml @@ -1,5 +1,4 @@ --- -version: "3" x-image: &image image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.9.1-rc.2} services: diff --git a/docker/docker-compose-with-azblob.yml b/docker/docker-compose-with-azblob.yml index 9fc76c4051817..e1bf11bb28ff0 100644 --- a/docker/docker-compose-with-azblob.yml +++ b/docker/docker-compose-with-azblob.yml @@ -1,5 +1,4 @@ --- -version: "3" x-image: &image image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.9.1-rc.2} services: diff --git a/docker/docker-compose-with-gcs.yml b/docker/docker-compose-with-gcs.yml index 36433d549c1c7..fcad2692f474d 100644 --- a/docker/docker-compose-with-gcs.yml +++ b/docker/docker-compose-with-gcs.yml @@ -1,5 +1,4 @@ --- -version: "3" x-image: &image image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.9.1-rc.2} services: diff --git a/docker/docker-compose-with-hdfs.yml b/docker/docker-compose-with-hdfs.yml index cf2b45078bac5..974cf922e77b4 100644 --- a/docker/docker-compose-with-hdfs.yml +++ b/docker/docker-compose-with-hdfs.yml @@ -1,5 +1,4 @@ --- -version: "3" services: compactor-0: image: ghcr.io/risingwavelabs/risingwave:RisingWave_1.6.1_HDFS_2.7-x86_64 diff --git a/docker/docker-compose-with-local-fs.yml b/docker/docker-compose-with-local-fs.yml index ab4545d649821..d7b5e22c36243 100644 --- a/docker/docker-compose-with-local-fs.yml +++ b/docker/docker-compose-with-local-fs.yml @@ -1,5 +1,4 @@ --- -version: "3" x-image: &image image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.9.1-rc.2} services: diff --git a/docker/docker-compose-with-obs.yml b/docker/docker-compose-with-obs.yml index b92d0cb077a16..45c84e32f652d 100644 --- a/docker/docker-compose-with-obs.yml +++ b/docker/docker-compose-with-obs.yml @@ -1,5 +1,4 @@ --- -version: "3" x-image: &image image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.9.1-rc.2} services: diff --git a/docker/docker-compose-with-oss.yml b/docker/docker-compose-with-oss.yml index adbfee86cc839..25aeae746e3b7 100644 --- a/docker/docker-compose-with-oss.yml +++ b/docker/docker-compose-with-oss.yml @@ -1,5 +1,4 @@ --- -version: "3" x-image: &image image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.9.1-rc.2} services: diff --git a/docker/docker-compose-with-s3.yml b/docker/docker-compose-with-s3.yml index eabe1f023ced7..75ffded8edcbc 100644 --- a/docker/docker-compose-with-s3.yml +++ b/docker/docker-compose-with-s3.yml @@ -1,5 +1,4 @@ --- -version: "3" x-image: &image image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.9.1-rc.2} services: diff --git a/docker/docker-compose-with-sqlite.yml b/docker/docker-compose-with-sqlite.yml index f65a4af7bc29f..065a60610f333 100644 --- a/docker/docker-compose-with-sqlite.yml +++ b/docker/docker-compose-with-sqlite.yml @@ -1,5 +1,4 @@ --- -version: "3" x-image: &image image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.9.1-rc.2} services: diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index 5ba67f78c3f56..6042fbba261df 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" x-image: &image image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.9.1-rc.2} services: diff --git a/integration_tests/ad-click/docker-compose.yml b/integration_tests/ad-click/docker-compose.yml index 62d5c3fb76517..2b84bb00d9950 100644 --- a/integration_tests/ad-click/docker-compose.yml +++ b/integration_tests/ad-click/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/ad-ctr/docker-compose.yml b/integration_tests/ad-ctr/docker-compose.yml index 0298f014db11a..2bec6c35295b8 100644 --- a/integration_tests/ad-ctr/docker-compose.yml +++ b/integration_tests/ad-ctr/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/big-query-sink/docker-compose.yml b/integration_tests/big-query-sink/docker-compose.yml index 6c93903df8bba..279f43a43b217 100644 --- a/integration_tests/big-query-sink/docker-compose.yml +++ b/integration_tests/big-query-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/cassandra-and-scylladb-sink/docker-compose.yml b/integration_tests/cassandra-and-scylladb-sink/docker-compose.yml index 0fa224ddab9d0..9f09b203ef700 100644 --- a/integration_tests/cassandra-and-scylladb-sink/docker-compose.yml +++ b/integration_tests/cassandra-and-scylladb-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: cassandra: image: cassandra:4.0 diff --git a/integration_tests/cdn-metrics/docker-compose.yml b/integration_tests/cdn-metrics/docker-compose.yml index 87adef35f8cf4..05d3d786e6279 100644 --- a/integration_tests/cdn-metrics/docker-compose.yml +++ b/integration_tests/cdn-metrics/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/citus-cdc/docker-compose.yml b/integration_tests/citus-cdc/docker-compose.yml index 6ce8341047ee4..8afb665e02cd1 100644 --- a/integration_tests/citus-cdc/docker-compose.yml +++ b/integration_tests/citus-cdc/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/clickhouse-sink/docker-compose.yml b/integration_tests/clickhouse-sink/docker-compose.yml index 1cf61ff8dfa30..beb2ee1254739 100644 --- a/integration_tests/clickhouse-sink/docker-compose.yml +++ b/integration_tests/clickhouse-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: clickhouse-server: image: clickhouse/clickhouse-server:23.3.8.21-alpine diff --git a/integration_tests/clickstream/docker-compose.yml b/integration_tests/clickstream/docker-compose.yml index 857c93f0d7577..4015a3a976ced 100644 --- a/integration_tests/clickstream/docker-compose.yml +++ b/integration_tests/clickstream/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/client-library/docker-compose.yml b/integration_tests/client-library/docker-compose.yml index c6868eaa42140..c8a03d353b18e 100644 --- a/integration_tests/client-library/docker-compose.yml +++ b/integration_tests/client-library/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/cockroach-sink/docker-compose.yml b/integration_tests/cockroach-sink/docker-compose.yml index b6b0c8d9e6c5f..d325c57865baf 100644 --- a/integration_tests/cockroach-sink/docker-compose.yml +++ b/integration_tests/cockroach-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/debezium-mysql/docker-compose.yml b/integration_tests/debezium-mysql/docker-compose.yml index 3462e5e3d09d1..6cb577ac23886 100644 --- a/integration_tests/debezium-mysql/docker-compose.yml +++ b/integration_tests/debezium-mysql/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/debezium-postgres/docker-compose.yml b/integration_tests/debezium-postgres/docker-compose.yml index c81c33fb3e455..327cb44d6db7c 100644 --- a/integration_tests/debezium-postgres/docker-compose.yml +++ b/integration_tests/debezium-postgres/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/debezium-sqlserver/docker-compose.yml b/integration_tests/debezium-sqlserver/docker-compose.yml index e88cb36e548b7..9d4bbbf0a5bb6 100644 --- a/integration_tests/debezium-sqlserver/docker-compose.yml +++ b/integration_tests/debezium-sqlserver/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/deltalake-sink/docker-compose.yml b/integration_tests/deltalake-sink/docker-compose.yml index 70b1e3c22e325..2a799f9fcf45b 100644 --- a/integration_tests/deltalake-sink/docker-compose.yml +++ b/integration_tests/deltalake-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: spark: image: apache/spark:3.3.1 diff --git a/integration_tests/doris-sink/docker-compose.yml b/integration_tests/doris-sink/docker-compose.yml index e1a7f1ef5e90e..4b43632f51319 100644 --- a/integration_tests/doris-sink/docker-compose.yml +++ b/integration_tests/doris-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: fe: platform: linux/amd64 diff --git a/integration_tests/elasticsearch-sink/docker-compose.yml b/integration_tests/elasticsearch-sink/docker-compose.yml index c885b7136a606..097de4beb5490 100644 --- a/integration_tests/elasticsearch-sink/docker-compose.yml +++ b/integration_tests/elasticsearch-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: elasticsearch7: image: docker.elastic.co/elasticsearch/elasticsearch:7.11.0 diff --git a/integration_tests/feature-store/docker-compose.yml b/integration_tests/feature-store/docker-compose.yml index 71633cce20a19..77de22a8c3522 100644 --- a/integration_tests/feature-store/docker-compose.yml +++ b/integration_tests/feature-store/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: kafka: image: confluentinc/cp-kafka:7.1.0 diff --git a/integration_tests/http-sink/docker-compose.yml b/integration_tests/http-sink/docker-compose.yml index 12546c4f5dd28..9a7c42b1443e0 100644 --- a/integration_tests/http-sink/docker-compose.yml +++ b/integration_tests/http-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/iceberg-sink/docker-compose.yml b/integration_tests/iceberg-sink/docker-compose.yml index 91cec5dd24430..84bda01b21ceb 100644 --- a/integration_tests/iceberg-sink/docker-compose.yml +++ b/integration_tests/iceberg-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" x-airflow-common: &airflow-common image: apache/airflow:2.6.2-python3.10 diff --git a/integration_tests/kafka-cdc-sink/docker-compose.yml b/integration_tests/kafka-cdc-sink/docker-compose.yml index 81f892354b8a0..1cebe9b73f284 100644 --- a/integration_tests/kafka-cdc-sink/docker-compose.yml +++ b/integration_tests/kafka-cdc-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/kafka-cdc/docker-compose.yml b/integration_tests/kafka-cdc/docker-compose.yml index f42c4399178d0..6eaa5b5ead7ab 100644 --- a/integration_tests/kafka-cdc/docker-compose.yml +++ b/integration_tests/kafka-cdc/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/kinesis-s3-source/docker-compose.yml b/integration_tests/kinesis-s3-source/docker-compose.yml index dc91e2095cbde..74dabde96f7ba 100644 --- a/integration_tests/kinesis-s3-source/docker-compose.yml +++ b/integration_tests/kinesis-s3-source/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/livestream/docker-compose.yml b/integration_tests/livestream/docker-compose.yml index 8dffce371562a..e263b704bc90d 100644 --- a/integration_tests/livestream/docker-compose.yml +++ b/integration_tests/livestream/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/mindsdb/docker-compose.yml b/integration_tests/mindsdb/docker-compose.yml index 40fe4e6192fa3..0cd82b10a6529 100644 --- a/integration_tests/mindsdb/docker-compose.yml +++ b/integration_tests/mindsdb/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/mongodb-cdc/docker-compose.yaml b/integration_tests/mongodb-cdc/docker-compose.yaml index eaf519b440569..de09a204d991b 100644 --- a/integration_tests/mongodb-cdc/docker-compose.yaml +++ b/integration_tests/mongodb-cdc/docker-compose.yaml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/mongodb/docker-compose.yaml b/integration_tests/mongodb/docker-compose.yaml index 59ac89215ec14..a2855c200e6b0 100644 --- a/integration_tests/mongodb/docker-compose.yaml +++ b/integration_tests/mongodb/docker-compose.yaml @@ -1,4 +1,3 @@ -version: "3" services: mongodb: image: mongo:4.4 diff --git a/integration_tests/mqtt/docker-compose.yml b/integration_tests/mqtt/docker-compose.yml index 04f73404be6aa..b91ddd482509c 100644 --- a/integration_tests/mqtt/docker-compose.yml +++ b/integration_tests/mqtt/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/mysql-cdc/docker-compose.yml b/integration_tests/mysql-cdc/docker-compose.yml index c0bba2ccc008b..b2779c42c05b6 100644 --- a/integration_tests/mysql-cdc/docker-compose.yml +++ b/integration_tests/mysql-cdc/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/mysql-sink/docker-compose.yml b/integration_tests/mysql-sink/docker-compose.yml index 3e1fc5544276f..8f8c4f9aa4336 100644 --- a/integration_tests/mysql-sink/docker-compose.yml +++ b/integration_tests/mysql-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/nats/docker-compose.yml b/integration_tests/nats/docker-compose.yml index 891c865744747..930f1a719fd7f 100644 --- a/integration_tests/nats/docker-compose.yml +++ b/integration_tests/nats/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/pinot-sink/docker-compose.yml b/integration_tests/pinot-sink/docker-compose.yml index fc4ad250880ce..c7d08dcc005e9 100644 --- a/integration_tests/pinot-sink/docker-compose.yml +++ b/integration_tests/pinot-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/postgres-cdc/docker-compose.yml b/integration_tests/postgres-cdc/docker-compose.yml index 7650da0779178..333ee2f4080c3 100644 --- a/integration_tests/postgres-cdc/docker-compose.yml +++ b/integration_tests/postgres-cdc/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/postgres-sink/docker-compose.yml b/integration_tests/postgres-sink/docker-compose.yml index 4d8638fdc3c07..6f5a16db64c24 100644 --- a/integration_tests/postgres-sink/docker-compose.yml +++ b/integration_tests/postgres-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/presto-trino/docker-compose.yml b/integration_tests/presto-trino/docker-compose.yml index a56135a4ae597..5de9dc34eb78a 100644 --- a/integration_tests/presto-trino/docker-compose.yml +++ b/integration_tests/presto-trino/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/prometheus/docker-compose.yml b/integration_tests/prometheus/docker-compose.yml index de3249df9253a..cd840807ea1ac 100644 --- a/integration_tests/prometheus/docker-compose.yml +++ b/integration_tests/prometheus/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/redis-sink/docker-compose.yml b/integration_tests/redis-sink/docker-compose.yml index dce27ae99895c..8f3c3eb9cdd85 100644 --- a/integration_tests/redis-sink/docker-compose.yml +++ b/integration_tests/redis-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: redis: image: 'redis:latest' diff --git a/integration_tests/schema-registry/docker-compose.yml b/integration_tests/schema-registry/docker-compose.yml index 80d4b90e4f7d2..2209fb7e20aee 100644 --- a/integration_tests/schema-registry/docker-compose.yml +++ b/integration_tests/schema-registry/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/starrocks-sink/docker-compose.yml b/integration_tests/starrocks-sink/docker-compose.yml index 70918713643d6..e3a06cc33587a 100644 --- a/integration_tests/starrocks-sink/docker-compose.yml +++ b/integration_tests/starrocks-sink/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: starrocks-fe: image: starrocks/fe-ubuntu:3.1.7 diff --git a/integration_tests/superset/docker-compose.yml b/integration_tests/superset/docker-compose.yml index 746a80fb9a064..3d7b9ed2494ca 100644 --- a/integration_tests/superset/docker-compose.yml +++ b/integration_tests/superset/docker-compose.yml @@ -9,7 +9,6 @@ x-superset-volumes: - ./docker:/app/docker - superset_home:/app/superset_home -version: "3.7" services: risingwave-standalone: extends: diff --git a/integration_tests/twitter-pulsar/docker-compose.yml b/integration_tests/twitter-pulsar/docker-compose.yml index d684be6b876a8..a3e91c9f8751a 100644 --- a/integration_tests/twitter-pulsar/docker-compose.yml +++ b/integration_tests/twitter-pulsar/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/twitter/docker-compose.yml b/integration_tests/twitter/docker-compose.yml index 37b2723cb8e50..e59e71b3839ce 100644 --- a/integration_tests/twitter/docker-compose.yml +++ b/integration_tests/twitter/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/integration_tests/upsert-avro/docker-compose.yml b/integration_tests/upsert-avro/docker-compose.yml index 291528f6fb319..9176ca053ba4e 100644 --- a/integration_tests/upsert-avro/docker-compose.yml +++ b/integration_tests/upsert-avro/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: @@ -53,4 +52,4 @@ volumes: external: false message_queue: external: false -name: risingwave-compose \ No newline at end of file +name: risingwave-compose diff --git a/integration_tests/vector/docker-compose.yml b/integration_tests/vector/docker-compose.yml index 4c2e6100b714a..13101925ac4a3 100644 --- a/integration_tests/vector/docker-compose.yml +++ b/integration_tests/vector/docker-compose.yml @@ -1,5 +1,4 @@ --- -version: "3" services: risingwave-standalone: extends: diff --git a/src/risedevtool/src/bin/risedev-compose.rs b/src/risedevtool/src/bin/risedev-compose.rs index 5ff56916deca6..89bb0592d0d85 100644 --- a/src/risedevtool/src/bin/risedev-compose.rs +++ b/src/risedevtool/src/bin/risedev-compose.rs @@ -245,7 +245,6 @@ fn main() -> Result<()> { } }); let compose_file = ComposeFile { - version: "3".into(), services: services.clone(), volumes: node_volumes, name: format!("risingwave-{}", opts.profile), @@ -303,7 +302,6 @@ fn main() -> Result<()> { } } let compose_file = ComposeFile { - version: "3".into(), services, volumes, name: format!("risingwave-{}", opts.profile), diff --git a/src/risedevtool/src/compose.rs b/src/risedevtool/src/compose.rs index 779ca23557622..a490560157527 100644 --- a/src/risedevtool/src/compose.rs +++ b/src/risedevtool/src/compose.rs @@ -56,7 +56,6 @@ pub struct HealthCheck { #[derive(Debug, Clone, Serialize)] pub struct ComposeFile { - pub version: String, pub services: BTreeMap, pub volumes: BTreeMap, pub name: String, From 13cdd9525791968624f93459e681b0ae2fd995ab Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 13 Jun 2024 11:06:54 +0800 Subject: [PATCH 30/37] feat(storage): remove magic number MAX_COMPACT_LEVEL_COUNT (#17219) --- proto/hummock.proto | 4 ++++ src/common/src/config.rs | 6 +++++- .../src/cmd_impl/hummock/compaction_group.rs | 4 ++++ src/ctl/src/lib.rs | 4 ++++ .../src/hummock/compaction/compaction_config.rs | 1 + .../picker/base_level_compaction_picker.rs | 1 + .../picker/compaction_task_validator.rs | 16 +++++----------- .../picker/intra_compaction_picker.rs | 1 + .../picker/min_overlap_compaction_picker.rs | 17 +++++++++++++++-- src/meta/src/hummock/compaction/picker/mod.rs | 2 -- .../compaction/picker/tier_compaction_picker.rs | 6 +----- .../hummock/manager/compaction_group_manager.rs | 4 +++- 12 files changed, 44 insertions(+), 22 deletions(-) diff --git a/proto/hummock.proto b/proto/hummock.proto index 2fc4c1a6d4b42..c4ac75f2fed64 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -706,6 +706,7 @@ message RiseCtlUpdateCompactionConfigRequest { bool enable_emergency_picker = 15; uint32 tombstone_reclaim_ratio = 16; CompressionAlgorithm compression_algorithm = 17; + uint32 max_l0_compact_level_count = 18; } } repeated uint64 compaction_group_ids = 1; @@ -885,6 +886,9 @@ message CompactionConfig { uint32 level0_overlapping_sub_level_compact_level_count = 18; uint32 tombstone_reclaim_ratio = 19; bool enable_emergency_picker = 20; + + // The limitation of the level count of l0 compaction + uint32 max_l0_compact_level_count = 21; } message TableStats { diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 6395cbf37ed15..6cad79849153a 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -1786,8 +1786,8 @@ pub mod default { const DEFAULT_MIN_OVERLAPPING_SUB_LEVEL_COMPACT_LEVEL_COUNT: u32 = 12; const DEFAULT_TOMBSTONE_RATIO_PERCENT: u32 = 40; const DEFAULT_EMERGENCY_PICKER: bool = true; - const DEFAULT_MAX_LEVEL: u32 = 6; + const DEFAULT_MAX_L0_COMPACT_LEVEL_COUNT: u32 = 42; use crate::catalog::hummock::CompactionFilterFlag; @@ -1854,6 +1854,10 @@ pub mod default { pub fn max_level() -> u32 { DEFAULT_MAX_LEVEL } + + pub fn max_l0_compact_level_count() -> u32 { + DEFAULT_MAX_L0_COMPACT_LEVEL_COUNT + } } pub mod object_store_config { diff --git a/src/ctl/src/cmd_impl/hummock/compaction_group.rs b/src/ctl/src/cmd_impl/hummock/compaction_group.rs index 3fb83e9d16cb3..d58aeb7bffe79 100644 --- a/src/ctl/src/cmd_impl/hummock/compaction_group.rs +++ b/src/ctl/src/cmd_impl/hummock/compaction_group.rs @@ -65,6 +65,7 @@ pub fn build_compaction_config_vec( enable_emergency_picker: Option, tombstone_reclaim_ratio: Option, compress_algorithm: Option, + max_l0_compact_level: Option, ) -> Vec { let mut configs = vec![]; if let Some(c) = max_bytes_for_level_base { @@ -115,6 +116,9 @@ pub fn build_compaction_config_vec( if let Some(c) = compress_algorithm { configs.push(MutableConfig::CompressionAlgorithm(c)) } + if let Some(c) = max_l0_compact_level { + configs.push(MutableConfig::MaxL0CompactLevelCount(c)) + } configs } diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index 1276d4bfce439..1f50250276d6e 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -259,6 +259,8 @@ enum HummockCommands { compression_level: Option, #[clap(long)] compression_algorithm: Option, + #[clap(long)] + max_l0_compact_level: Option, }, /// Split given compaction group into two. Moves the given tables to the new group. SplitCompactionGroup { @@ -690,6 +692,7 @@ async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { tombstone_reclaim_ratio, compression_level, compression_algorithm, + max_l0_compact_level, }) => { cmd_impl::hummock::update_compaction_config( context, @@ -719,6 +722,7 @@ async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { } else { None }, + max_l0_compact_level, ), ) .await? diff --git a/src/meta/src/hummock/compaction/compaction_config.rs b/src/meta/src/hummock/compaction/compaction_config.rs index e6750084432ec..de91bf4f79ded 100644 --- a/src/meta/src/hummock/compaction/compaction_config.rs +++ b/src/meta/src/hummock/compaction/compaction_config.rs @@ -64,6 +64,7 @@ impl CompactionConfigBuilder { compaction_config::level0_overlapping_sub_level_compact_level_count(), tombstone_reclaim_ratio: compaction_config::tombstone_reclaim_ratio(), enable_emergency_picker: compaction_config::enable_emergency_picker(), + max_l0_compact_level_count: compaction_config::max_l0_compact_level_count(), }, } } diff --git a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs index f98e14203d95b..a2c3a7d52802e 100644 --- a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs @@ -166,6 +166,7 @@ impl LevelCompactionPicker { self.config.level0_max_compact_file_number, overlap_strategy.clone(), self.developer_config.enable_check_task_level_overlap, + self.config.max_l0_compact_level_count as usize, ); let mut max_vnode_partition_idx = 0; diff --git a/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs b/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs index 29119ae283b0a..c7dd27a6b1907 100644 --- a/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs +++ b/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use risingwave_pb::hummock::CompactionConfig; -use super::{CompactionInput, LocalPickerStatistic, MAX_COMPACT_LEVEL_COUNT}; +use super::{CompactionInput, LocalPickerStatistic}; #[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] pub enum ValidationRuleType { @@ -89,14 +89,8 @@ struct TierCompactionTaskValidationRule { impl CompactionTaskValidationRule for TierCompactionTaskValidationRule { fn validate(&self, input: &CompactionInput, stats: &mut LocalPickerStatistic) -> bool { - // Limit sstable file count to avoid using too much memory. - let overlapping_max_compact_file_numer = std::cmp::min( - self.config.level0_max_compact_file_number, - MAX_COMPACT_LEVEL_COUNT as u64, - ); - - if input.total_file_count >= overlapping_max_compact_file_numer - || input.input_levels.len() >= MAX_COMPACT_LEVEL_COUNT + if input.total_file_count >= self.config.level0_max_compact_file_number + || input.input_levels.len() >= self.config.max_l0_compact_level_count as usize { return true; } @@ -130,7 +124,7 @@ impl CompactionTaskValidationRule for IntraCompactionTaskValidationRule { fn validate(&self, input: &CompactionInput, stats: &mut LocalPickerStatistic) -> bool { if (input.total_file_count >= self.config.level0_max_compact_file_number && input.input_levels.len() > 1) - || input.input_levels.len() >= MAX_COMPACT_LEVEL_COUNT + || input.input_levels.len() >= self.config.max_l0_compact_level_count as usize { return true; } @@ -178,7 +172,7 @@ struct BaseCompactionTaskValidationRule { impl CompactionTaskValidationRule for BaseCompactionTaskValidationRule { fn validate(&self, input: &CompactionInput, stats: &mut LocalPickerStatistic) -> bool { if input.total_file_count >= self.config.level0_max_compact_file_number - || input.input_levels.len() >= MAX_COMPACT_LEVEL_COUNT + || input.input_levels.len() >= self.config.max_l0_compact_level_count as usize { return true; } diff --git a/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs index 993ad79d59b2e..6b5dcae7d0c31 100644 --- a/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs @@ -144,6 +144,7 @@ impl IntraCompactionPicker { self.config.level0_max_compact_file_number, overlap_strategy.clone(), self.developer_config.enable_check_task_level_overlap, + self.config.max_l0_compact_level_count as usize, ); let l0_select_tables_vec = non_overlap_sub_level_picker diff --git a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs index a0d896daa439c..57dd5469d42ae 100644 --- a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs @@ -20,7 +20,7 @@ use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::{InputLevel, Level, LevelType, SstableInfo}; -use super::{CompactionInput, CompactionPicker, LocalPickerStatistic, MAX_COMPACT_LEVEL_COUNT}; +use super::{CompactionInput, CompactionPicker, LocalPickerStatistic}; use crate::hummock::compaction::overlap_strategy::OverlapStrategy; use crate::hummock::level_handler::LevelHandler; @@ -197,6 +197,7 @@ impl NonOverlapSubLevelPicker { max_file_count: u64, overlap_strategy: Arc, enable_check_task_level_overlap: bool, + max_expected_level_count: usize, ) -> Self { Self { min_compaction_bytes, @@ -205,7 +206,7 @@ impl NonOverlapSubLevelPicker { max_file_count, overlap_strategy, enable_check_task_level_overlap, - max_expected_level_count: MAX_COMPACT_LEVEL_COUNT, + max_expected_level_count, } } @@ -533,6 +534,8 @@ impl NonOverlapSubLevelPicker { pub mod tests { use std::collections::BTreeSet; + use risingwave_common::config::default::compaction_config; + use super::*; use crate::hummock::compaction::overlap_strategy::RangeOverlapStrategy; use crate::hummock::compaction::selector::tests::{ @@ -736,6 +739,7 @@ pub mod tests { 10000, Arc::new(RangeOverlapStrategy::default()), true, + compaction_config::max_l0_compact_level_count() as usize, ); let ret = picker.pick_l0_multi_non_overlap_level(&levels, &levels_handlers[0]); assert_eq!(6, ret.len()); @@ -750,6 +754,7 @@ pub mod tests { 10000, Arc::new(RangeOverlapStrategy::default()), true, + compaction_config::max_l0_compact_level_count() as usize, ); let ret = picker.pick_l0_multi_non_overlap_level(&levels, &levels_handlers[0]); assert_eq!(6, ret.len()); @@ -764,6 +769,7 @@ pub mod tests { 5, Arc::new(RangeOverlapStrategy::default()), true, + compaction_config::max_l0_compact_level_count() as usize, ); let ret = picker.pick_l0_multi_non_overlap_level(&levels, &levels_handlers[0]); assert_eq!(6, ret.len()); @@ -839,6 +845,7 @@ pub mod tests { 10000, Arc::new(RangeOverlapStrategy::default()), true, + compaction_config::max_l0_compact_level_count() as usize, ); let ret = picker.pick_l0_multi_non_overlap_level(&levels, &levels_handlers[0]); assert_eq!(6, ret.len()); @@ -854,6 +861,7 @@ pub mod tests { 10000, Arc::new(RangeOverlapStrategy::default()), true, + compaction_config::max_l0_compact_level_count() as usize, ); let ret = picker.pick_l0_multi_non_overlap_level(&levels, &levels_handlers[0]); assert_eq!(6, ret.len()); @@ -869,6 +877,7 @@ pub mod tests { max_file_count, Arc::new(RangeOverlapStrategy::default()), true, + compaction_config::max_l0_compact_level_count() as usize, ); let ret = picker.pick_l0_multi_non_overlap_level(&levels, &levels_handlers[0]); assert_eq!(6, ret.len()); @@ -892,6 +901,7 @@ pub mod tests { 10000, Arc::new(RangeOverlapStrategy::default()), true, + compaction_config::max_l0_compact_level_count() as usize, ); let ret = picker.pick_l0_multi_non_overlap_level(&levels, &levels_handlers[0]); assert_eq!(3, ret.len()); @@ -1019,6 +1029,7 @@ pub mod tests { 10000, Arc::new(RangeOverlapStrategy::default()), true, + compaction_config::max_l0_compact_level_count() as usize, ); let ret = picker.pick_l0_multi_non_overlap_level(&levels, &levels_handlers[0]); { @@ -1036,6 +1047,7 @@ pub mod tests { 10000, Arc::new(RangeOverlapStrategy::default()), true, + compaction_config::max_l0_compact_level_count() as usize, ); let ret = picker.pick_l0_multi_non_overlap_level(&levels, &levels_handlers[0]); { @@ -1053,6 +1065,7 @@ pub mod tests { 3, Arc::new(RangeOverlapStrategy::default()), true, + compaction_config::max_l0_compact_level_count() as usize, ); let ret = picker.pick_l0_multi_non_overlap_level(&levels, &levels_handlers[0]); { diff --git a/src/meta/src/hummock/compaction/picker/mod.rs b/src/meta/src/hummock/compaction/picker/mod.rs index f6dc46c99106c..6d464b9a33bcd 100644 --- a/src/meta/src/hummock/compaction/picker/mod.rs +++ b/src/meta/src/hummock/compaction/picker/mod.rs @@ -43,8 +43,6 @@ pub use ttl_reclaim_compaction_picker::{TtlPickerState, TtlReclaimCompactionPick use crate::hummock::level_handler::LevelHandler; -pub const MAX_COMPACT_LEVEL_COUNT: usize = 42; - #[derive(Default, Debug)] pub struct LocalPickerStatistic { pub skip_by_write_amp_limit: u64, diff --git a/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs index 9ed22ba551fcc..ce86b523f6e86 100644 --- a/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs @@ -21,7 +21,6 @@ use super::{ CompactionInput, CompactionPicker, CompactionTaskValidator, LocalPickerStatistic, ValidationRuleType, }; -use crate::hummock::compaction::picker::MAX_COMPACT_LEVEL_COUNT; use crate::hummock::level_handler::LevelHandler; pub struct TierCompactionPicker { @@ -87,10 +86,7 @@ impl TierCompactionPicker { let mut compaction_bytes = level.total_file_size; let mut compact_file_count = level.table_infos.len() as u64; // Limit sstable file count to avoid using too much memory. - let overlapping_max_compact_file_numer = std::cmp::min( - self.config.level0_max_compact_file_number, - MAX_COMPACT_LEVEL_COUNT as u64, - ); + let overlapping_max_compact_file_numer = self.config.level0_max_compact_file_number; for other in &l0.sub_levels[idx + 1..] { if compaction_bytes > max_compaction_bytes { diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index df5189d539180..063ce4e0a58b1 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -885,11 +885,13 @@ fn update_compaction_config(target: &mut CompactionConfig, items: &[MutableConfi MutableConfig::TombstoneReclaimRatio(c) => { target.tombstone_reclaim_ratio = *c; } - MutableConfig::CompressionAlgorithm(c) => { target.compression_algorithm[c.get_level() as usize] .clone_from(&c.compression_algorithm); } + MutableConfig::MaxL0CompactLevelCount(c) => { + target.max_l0_compact_level_count = *c; + } } } } From 32a1129da3a9d84d201ffb9a01b89976889e3a76 Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 13 Jun 2024 14:18:46 +0800 Subject: [PATCH 31/37] fix(compactor): Compactor potential oom risk of builder (#16802) --- proto/hummock.proto | 1 + src/common/src/config.rs | 55 ++++++++++++++--- src/config/docs.md | 3 +- src/meta/src/hummock/compactor_manager.rs | 7 +++ src/meta/src/hummock/manager/compaction.rs | 61 +++++++++---------- src/meta/src/hummock/manager/timer_task.rs | 17 ++++-- src/storage/benches/bench_multi_builder.rs | 47 ++++++++------ src/storage/hummock_sdk/src/compact.rs | 11 ++-- .../src/hummock/compactor/compactor_runner.rs | 1 - .../compactor/fast_compactor_runner.rs | 19 +++--- src/storage/src/hummock/compactor/mod.rs | 48 +++++---------- .../src/hummock/sstable/multi_builder.rs | 50 +++++++++++---- src/storage/src/opts.rs | 5 ++ 13 files changed, 202 insertions(+), 123 deletions(-) diff --git a/proto/hummock.proto b/proto/hummock.proto index c4ac75f2fed64..89a0438fc43a7 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -340,6 +340,7 @@ message CompactTask { JOIN_HANDLE_FAILED = 11; TRACK_SST_OBJECT_ID_FAILED = 12; NO_AVAIL_CPU_RESOURCE_CANCELED = 13; + HEARTBEAT_PROGRESS_CANCELED = 14; } // SSTs to be compacted, which will be removed from LSM after compaction repeated InputLevel input_ssts = 1; diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 6cad79849153a..d8bb65e90b062 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -763,6 +763,14 @@ pub struct StorageConfig { #[serde(default = "default::storage::mem_table_spill_threshold")] pub mem_table_spill_threshold: usize, + /// The concurrent uploading number of `SSTables` of buidler + #[serde(default = "default::storage::compactor_concurrent_uploading_sst_count")] + pub compactor_concurrent_uploading_sst_count: Option, + + /// Object storage configuration + /// 1. General configuration + /// 2. Some special configuration of Backend + /// 3. Retry and timeout configuration #[serde(default)] pub object_store: ObjectStoreConfig, } @@ -1024,9 +1032,13 @@ pub struct ObjectStoreConfig { #[serde(default = "default::object_store_config::object_store_set_atomic_write_dir")] pub object_store_set_atomic_write_dir: bool, + /// Retry and timeout configuration + /// Description retry strategy driven by exponential back-off + /// Exposes the timeout and retries of each Object store interface. Therefore, the total timeout for each interface is determined based on the interface's timeout/retry configuration and the exponential back-off policy. #[serde(default)] pub retry: ObjectStoreRetryConfig, + /// Some special configuration of S3 Backend #[serde(default)] pub s3: S3ObjectStoreConfig, } @@ -1080,66 +1092,89 @@ pub struct S3ObjectStoreDeveloperConfig { #[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] pub struct ObjectStoreRetryConfig { + // A retry strategy driven by exponential back-off. + // The retry strategy is used for all object store operations. + /// Given a base duration for retry strategy in milliseconds. #[serde(default = "default::object_store_config::object_store_req_backoff_interval_ms")] pub req_backoff_interval_ms: u64, + + /// The max delay interval for the retry strategy. No retry delay will be longer than this `Duration`. #[serde(default = "default::object_store_config::object_store_req_backoff_max_delay_ms")] pub req_backoff_max_delay_ms: u64, + + /// A multiplicative factor that will be applied to the exponential back-off retry delay. #[serde(default = "default::object_store_config::object_store_req_backoff_factor")] pub req_backoff_factor: u64, - // upload + /// Maximum timeout for `upload` operation #[serde(default = "default::object_store_config::object_store_upload_attempt_timeout_ms")] pub upload_attempt_timeout_ms: u64, + + /// Total counts of `upload` operation retries #[serde(default = "default::object_store_config::object_store_upload_retry_attempts")] pub upload_retry_attempts: usize, - // streaming_upload_init + streaming_upload + /// Maximum timeout for `streaming_upload_init` and `streaming_upload` #[serde( default = "default::object_store_config::object_store_streaming_upload_attempt_timeout_ms" )] pub streaming_upload_attempt_timeout_ms: u64, + + /// Total counts of `streaming_upload` operation retries #[serde( default = "default::object_store_config::object_store_streaming_upload_retry_attempts" )] pub streaming_upload_retry_attempts: usize, - // read + /// Maximum timeout for `read` operation #[serde(default = "default::object_store_config::object_store_read_attempt_timeout_ms")] pub read_attempt_timeout_ms: u64, + + /// Total counts of `read` operation retries #[serde(default = "default::object_store_config::object_store_read_retry_attempts")] pub read_retry_attempts: usize, - // streaming_read_init + streaming_read + /// Maximum timeout for `streaming_read_init` and `streaming_read` operation #[serde( default = "default::object_store_config::object_store_streaming_read_attempt_timeout_ms" )] pub streaming_read_attempt_timeout_ms: u64, + + /// Total counts of `streaming_read operation` retries #[serde(default = "default::object_store_config::object_store_streaming_read_retry_attempts")] pub streaming_read_retry_attempts: usize, - // metadata + /// Maximum timeout for `metadata` operation #[serde(default = "default::object_store_config::object_store_metadata_attempt_timeout_ms")] pub metadata_attempt_timeout_ms: u64, + + /// Total counts of `metadata` operation retries #[serde(default = "default::object_store_config::object_store_metadata_retry_attempts")] pub metadata_retry_attempts: usize, - // delete + /// Maximum timeout for `delete` operation #[serde(default = "default::object_store_config::object_store_delete_attempt_timeout_ms")] pub delete_attempt_timeout_ms: u64, + + /// Total counts of `delete` operation retries #[serde(default = "default::object_store_config::object_store_delete_retry_attempts")] pub delete_retry_attempts: usize, - // delete_object + /// Maximum timeout for `delete_object` operation #[serde( default = "default::object_store_config::object_store_delete_objects_attempt_timeout_ms" )] pub delete_objects_attempt_timeout_ms: u64, + + /// Total counts of `delete_object` operation retries #[serde(default = "default::object_store_config::object_store_delete_objects_retry_attempts")] pub delete_objects_retry_attempts: usize, - // list + /// Maximum timeout for `list` operation #[serde(default = "default::object_store_config::object_store_list_attempt_timeout_ms")] pub list_attempt_timeout_ms: u64, + + /// Total counts of `list` operation retries #[serde(default = "default::object_store_config::object_store_list_retry_attempts")] pub list_retry_attempts: usize, } @@ -1509,6 +1544,10 @@ pub mod default { pub fn max_prefetch_block_number() -> usize { 16 } + + pub fn compactor_concurrent_uploading_sst_count() -> Option { + None + } } pub mod streaming { diff --git a/src/config/docs.md b/src/config/docs.md index 540d86085b342..ab33559260162 100644 --- a/src/config/docs.md +++ b/src/config/docs.md @@ -105,6 +105,7 @@ This page is automatically generated by `./risedev generate-example-config` | cache_refill | | | | check_compaction_result | | false | | compact_iter_recreate_timeout_ms | | 600000 | +| compactor_concurrent_uploading_sst_count | The concurrent uploading number of `SSTables` of buidler | | | compactor_fast_max_compact_delete_ratio | | 40 | | compactor_fast_max_compact_task_size | | 2147483648 | | compactor_iter_max_io_retry_times | | 8 | @@ -128,7 +129,7 @@ This page is automatically generated by `./risedev generate-example-config` | meta_file_cache | | | | min_sst_size_for_streaming_upload | Whether to enable streaming upload for sstable. | 33554432 | | min_sstable_size_mb | | 32 | -| object_store | | | +| object_store | Object storage configuration 1. General configuration 2. Some special configuration of Backend 3. Retry and timeout configuration | | | prefetch_buffer_capacity_mb | max memory usage for large query | | | share_buffer_compaction_worker_threads_number | Worker threads number of dedicated tokio runtime for share buffer compaction. 0 means use tokio's default value (number of CPU core). | 4 | | share_buffer_upload_concurrency | Number of tasks shared buffer can upload in parallel. | 8 | diff --git a/src/meta/src/hummock/compactor_manager.rs b/src/meta/src/hummock/compactor_manager.rs index d9b5a481f420a..0876f31d211e3 100644 --- a/src/meta/src/hummock/compactor_manager.rs +++ b/src/meta/src/hummock/compactor_manager.rs @@ -101,6 +101,13 @@ impl Compactor { Ok(()) } + pub fn cancel_tasks(&self, task_ids: &Vec) -> MetaResult<()> { + for task_id in task_ids { + self.cancel_task(*task_id)?; + } + Ok(()) + } + pub fn context_id(&self) -> HummockContextId { self.context_id } diff --git a/src/meta/src/hummock/manager/compaction.rs b/src/meta/src/hummock/manager/compaction.rs index cc5c878a8baef..7a27061c8d373 100644 --- a/src/meta/src/hummock/manager/compaction.rs +++ b/src/meta/src/hummock/manager/compaction.rs @@ -105,6 +105,7 @@ static CANCEL_STATUS_SET: LazyLock> = LazyLock::new(|| { TaskStatus::InvalidGroupCanceled, TaskStatus::NoAvailMemoryResourceCanceled, TaskStatus::NoAvailCpuResourceCanceled, + TaskStatus::HeartbeatProgressCanceled, ] .into_iter() .collect() @@ -464,40 +465,36 @@ impl HummockManager { progress, }) => { let compactor_manager = hummock_manager.compactor_manager.clone(); - let cancel_tasks = compactor_manager.update_task_heartbeats(&progress); - if let Some(compactor) = compactor_manager.get_compactor(context_id) { - // TODO: task cancellation can be batched - for task in cancel_tasks { - tracing::info!( - "Task with group_id {} task_id {} with context_id {} has expired due to lack of visible progress", - task.compaction_group_id, - task.task_id, - context_id, - ); + let cancel_tasks = compactor_manager.update_task_heartbeats(&progress).into_iter().map(|task|task.task_id).collect::>(); + if !cancel_tasks.is_empty() { + tracing::info!( + "Tasks cancel with task_ids {:?} with context_id {} has expired due to lack of visible progress", + cancel_tasks, + context_id, + ); - if let Err(e) = - hummock_manager - .cancel_compact_task(task.task_id, TaskStatus::HeartbeatCanceled) - .await - { - tracing::error!( - task_id = task.task_id, - error = %e.as_report(), - "Attempt to remove compaction task due to elapsed heartbeat failed. We will continue to track its heartbeat - until we can successfully report its status." - ); - } - - // Forcefully cancel the task so that it terminates - // early on the compactor - // node. - let _ = compactor.cancel_task(task.task_id); - tracing::info!( - "CancelTask operation for task_id {} has been sent to node with context_id {}", - context_id, - task.task_id + if let Err(e) = hummock_manager + .cancel_compact_tasks(cancel_tasks.clone(), TaskStatus::HeartbeatProgressCanceled) + .await + { + tracing::error!( + error = %e.as_report(), + "Attempt to remove compaction task due to elapsed heartbeat failed. We will continue to track its heartbeat + until we can successfully report its status." ); } + } + + if let Some(compactor) = compactor_manager.get_compactor(context_id) { + // Forcefully cancel the task so that it terminates + // early on the compactor + // node. + let _ = compactor.cancel_tasks(&cancel_tasks); + tracing::info!( + "CancelTask operation for task_id {:?} has been sent to node with context_id {}", + cancel_tasks, + context_id + ); } else { // Determine the validity of the compactor streaming rpc. When the compactor no longer exists in the manager, the stream will be removed. // Tip: Connectivity to the compactor will be determined through the `send_event` operation. When send fails, it will be removed from the manager @@ -1004,7 +1001,7 @@ impl HummockManager { Ok(ret[0]) } - async fn cancel_compact_tasks( + pub async fn cancel_compact_tasks( &self, tasks: Vec, task_status: TaskStatus, diff --git a/src/meta/src/hummock/manager/timer_task.rs b/src/meta/src/hummock/manager/timer_task.rs index e28a2d2e9fb02..6b68950a28e3b 100644 --- a/src/meta/src/hummock/manager/timer_task.rs +++ b/src/meta/src/hummock/manager/timer_task.rs @@ -257,16 +257,25 @@ impl HummockManager { // progress (meta + compactor) // 2. meta periodically scans the task and performs a cancel on // the meta side for tasks that are not updated by heartbeat - for task in compactor_manager.get_heartbeat_expired_tasks() { + let expired_tasks: Vec = compactor_manager + .get_heartbeat_expired_tasks() + .into_iter() + .map(|task| task.task_id) + .collect(); + if !expired_tasks.is_empty() { + tracing::info!( + expired_tasks = ?expired_tasks, + "Heartbeat expired compaction tasks detected. Attempting to cancel tasks.", + ); if let Err(e) = hummock_manager - .cancel_compact_task( - task.task_id, + .cancel_compact_tasks( + expired_tasks.clone(), TaskStatus::HeartbeatCanceled, ) .await { tracing::error!( - task_id = task.task_id, + expired_tasks = ?expired_tasks, error = %e.as_report(), "Attempt to remove compaction task due to elapsed heartbeat failed. We will continue to track its heartbeat until we can successfully report its status", diff --git a/src/storage/benches/bench_multi_builder.rs b/src/storage/benches/bench_multi_builder.rs index 08318891eac8c..b67be3467c48a 100644 --- a/src/storage/benches/bench_multi_builder.rs +++ b/src/storage/benches/bench_multi_builder.rs @@ -21,8 +21,6 @@ use std::time::Duration; use criterion::{criterion_group, criterion_main, Criterion}; use foyer::HybridCacheBuilder; -use futures::future::try_join_all; -use itertools::Itertools; use rand::random; use risingwave_common::catalog::TableId; use risingwave_common::config::{MetricLevel, ObjectStoreConfig}; @@ -118,17 +116,14 @@ async fn build_tables( .await .unwrap(); } - let split_table_outputs = builder.finish().await.unwrap(); - let ssts = split_table_outputs - .iter() - .map(|handle| handle.sst_info.sst_info.clone()) - .collect_vec(); - let join_handles = split_table_outputs + + builder + .finish() + .await + .unwrap() .into_iter() - .map(|o| o.upload_join_handle) - .collect_vec(); - try_join_all(join_handles).await.unwrap(); - ssts + .map(|info| info.sst_info) + .collect() } async fn generate_sstable_store(object_store: Arc) -> Arc { @@ -160,9 +155,9 @@ async fn generate_sstable_store(object_store: Arc) -> Arc u64 { let mut result = 0; // When building the SstableStreamIterator, sstable_syncable will fetch the SstableMeta and seek @@ -223,12 +222,10 @@ pub fn estimate_memory_for_compact_task( // output // builder will maintain SstableInfo + block_builder(block) + writer (block to vec) let estimated_meta_size = sst_capacity * task_max_sst_meta_ratio / 100; - if support_streaming_upload { - result += estimated_meta_size + 2 * block_size - } else { - result += estimated_meta_size + sst_capacity; // Use sst_capacity to avoid BatchUploader - // memory bursts. - } + + // FIXME: sst_capacity is the upper bound of the memory usage of the streaming sstable uploader + // A more reasonable memory limit method needs to be adopted, this is just a temporary fix. + result += estimated_meta_size + sst_capacity; result } diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 05b3d7dd182d9..41a55518158de 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -383,7 +383,6 @@ pub async fn compact( .object_store_recv_buffer_size .unwrap_or(6 * 1024 * 1024) as u64, capacity as u64, - context.sstable_store.store().support_streaming_upload(), ) * compact_task.splits.len() as u64; tracing::info!( diff --git a/src/storage/src/hummock/compactor/fast_compactor_runner.rs b/src/storage/src/hummock/compactor/fast_compactor_runner.rs index 7a30a6fd2ef65..81f0210d945c1 100644 --- a/src/storage/src/hummock/compactor/fast_compactor_runner.rs +++ b/src/storage/src/hummock/compactor/fast_compactor_runner.rs @@ -316,6 +316,9 @@ impl CompactorRunner { context.compactor_metrics.clone(), Some(task_progress.clone()), task_config.table_vnode_partition.clone(), + context + .storage_opts + .compactor_concurrent_uploading_sst_count, ); assert_eq!( task.input_ssts.len(), @@ -505,17 +508,19 @@ impl CompactorRunner { ); let statistic = self.executor.take_statistics(); - let outputs = self.executor.builder.finish().await?; - let ssts = Compactor::report_progress( + let output_ssts = self.executor.builder.finish().await?; + Compactor::report_progress( self.metrics.clone(), Some(self.executor.task_progress.clone()), - outputs, + &output_ssts, false, - ) - .await?; - let sst_infos = ssts.iter().map(|sst| sst.sst_info.clone()).collect_vec(); + ); + let sst_infos = output_ssts + .iter() + .map(|sst| sst.sst_info.clone()) + .collect_vec(); assert!(can_concat(&sst_infos)); - Ok((ssts, statistic)) + Ok((output_ssts, statistic)) } } diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index cddaffc584f66..898d91d530b22 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -83,11 +83,10 @@ use crate::filter_key_extractor::{ use crate::hummock::compactor::compaction_utils::calculate_task_parallelism; use crate::hummock::compactor::compactor_runner::{compact_and_build_sst, compact_done}; use crate::hummock::iterator::{Forward, HummockIterator}; -use crate::hummock::multi_builder::SplitTableOutput; use crate::hummock::vacuum::Vacuum; use crate::hummock::{ - validate_ssts, BlockedXor16FilterBuilder, FilterBuilder, HummockError, - SharedComapctorObjectIdManager, SstableWriterFactory, UnifiedSstableWriterFactory, + validate_ssts, BlockedXor16FilterBuilder, FilterBuilder, SharedComapctorObjectIdManager, + SstableWriterFactory, UnifiedSstableWriterFactory, }; use crate::monitor::CompactorMetrics; @@ -175,20 +174,19 @@ impl Compactor { compact_timer.observe_duration(); - let ssts = Self::report_progress( + Self::report_progress( self.context.compactor_metrics.clone(), task_progress, - split_table_outputs, + &split_table_outputs, self.context.is_share_buffer_compact, - ) - .await?; + ); self.context .compactor_metrics .get_table_id_total_time_duration .observe(self.get_id_time.load(Ordering::Relaxed) as f64 / 1000.0 / 1000.0); - debug_assert!(ssts + debug_assert!(split_table_outputs .iter() .all(|table_info| table_info.sst_info.get_table_ids().is_sorted())); @@ -198,33 +196,20 @@ impl Compactor { "Finish Task {:?} split_index {:?} sst count {}", task_id, split_index, - ssts.len() + split_table_outputs.len() ); } - Ok((ssts, table_stats_map)) + Ok((split_table_outputs, table_stats_map)) } - pub async fn report_progress( + pub fn report_progress( metrics: Arc, task_progress: Option>, - split_table_outputs: Vec, + ssts: &Vec, is_share_buffer_compact: bool, - ) -> HummockResult> { - let mut ssts = Vec::with_capacity(split_table_outputs.len()); - let mut rets = vec![]; - - for SplitTableOutput { - sst_info, - upload_join_handle, - } in split_table_outputs - { + ) { + for sst_info in ssts { let sst_size = sst_info.file_size(); - ssts.push(sst_info); - let ret = upload_join_handle - .verbose_instrument_await("upload") - .await - .map_err(HummockError::sstable_upload_error); - rets.push(ret); if let Some(tracker) = &task_progress { tracker.inc_ssts_uploaded(); tracker.dec_num_pending_write_io(); @@ -235,10 +220,6 @@ impl Compactor { metrics.compaction_upload_sst_counts.inc(); } } - for ret in rets { - ret??; - } - Ok(ssts) } async fn compact_key_range_impl( @@ -249,7 +230,7 @@ impl Compactor { filter_key_extractor: Arc, task_progress: Option>, object_id_getter: Box, - ) -> HummockResult<(Vec, CompactionStatistics)> { + ) -> HummockResult<(Vec, CompactionStatistics)> { let builder_factory = RemoteBuilderFactory:: { object_id_getter, limiter: self.context.memory_limiter.clone(), @@ -266,6 +247,9 @@ impl Compactor { self.context.compactor_metrics.clone(), task_progress.clone(), self.task_config.table_vnode_partition.clone(), + self.context + .storage_opts + .compactor_concurrent_uploading_sst_count, ); let compaction_statistics = compact_and_build_sst( &mut sst_builder, diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index 7354cb89c81fc..4e364ce9f94f5 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -18,6 +18,8 @@ use std::sync::atomic::Ordering::SeqCst; use std::sync::Arc; use bytes::Bytes; +use futures::stream::FuturesUnordered; +use futures::StreamExt; use num_integer::Integer; use risingwave_common::hash::VirtualNode; use risingwave_hummock_sdk::key::{FullKey, UserKey}; @@ -29,8 +31,8 @@ use crate::hummock::sstable::filter::FilterBuilder; use crate::hummock::sstable_store::SstableStoreRef; use crate::hummock::value::HummockValue; use crate::hummock::{ - BatchUploadWriter, BlockMeta, CachePolicy, HummockResult, MemoryLimiter, SstableBuilder, - SstableBuilderOptions, SstableWriter, SstableWriterOptions, Xor16FilterBuilder, + BatchUploadWriter, BlockMeta, CachePolicy, HummockError, HummockResult, MemoryLimiter, + SstableBuilder, SstableBuilderOptions, SstableWriter, SstableWriterOptions, Xor16FilterBuilder, }; use crate::monitor::CompactorMetrics; @@ -43,11 +45,6 @@ pub trait TableBuilderFactory { async fn open_builder(&mut self) -> HummockResult>; } -pub struct SplitTableOutput { - pub sst_info: LocalSstableInfo, - pub upload_join_handle: UploadJoinHandle, -} - /// A wrapper for [`SstableBuilder`] which automatically split key-value pairs into multiple tables, /// based on their target capacity set in options. /// @@ -59,7 +56,7 @@ where /// When creating a new [`SstableBuilder`], caller use this factory to generate it. builder_factory: F, - sst_outputs: Vec, + sst_outputs: Vec, current_builder: Option>, @@ -75,6 +72,10 @@ where /// When vnode of the coming key is greater than `largest_vnode_in_current_partition`, we will /// switch SST. largest_vnode_in_current_partition: usize, + + concurrent_upload_join_handle: FuturesUnordered, + + concurrent_uploading_sst_count: Option, } impl CapacitySplitTableBuilder @@ -88,6 +89,7 @@ where compactor_metrics: Arc, task_progress: Option>, table_partition_vnode: BTreeMap, + concurrent_uploading_sst_count: Option, ) -> Self { Self { builder_factory, @@ -99,6 +101,8 @@ where table_partition_vnode, split_weight_by_vnode: 0, largest_vnode_in_current_partition: VirtualNode::MAX.to_index(), + concurrent_upload_join_handle: FuturesUnordered::new(), + concurrent_uploading_sst_count, } } @@ -113,6 +117,8 @@ where table_partition_vnode: BTreeMap::default(), split_weight_by_vnode: 0, largest_vnode_in_current_partition: VirtualNode::MAX.to_index(), + concurrent_upload_join_handle: FuturesUnordered::new(), + concurrent_uploading_sst_count: None, } } @@ -264,6 +270,7 @@ where /// If there's no builder created, or current one is already sealed before, then this function /// will be no-op. pub async fn seal_current(&mut self) -> HummockResult<()> { + use await_tree::InstrumentAwait; if let Some(builder) = self.current_builder.take() { let builder_output = builder.finish().await?; { @@ -302,17 +309,33 @@ where .observe(builder_output.epoch_count as _); } } - self.sst_outputs.push(SplitTableOutput { - upload_join_handle: builder_output.writer_output, - sst_info: builder_output.sst_info, - }); + + self.concurrent_upload_join_handle + .push(builder_output.writer_output); + + self.sst_outputs.push(builder_output.sst_info); + + if let Some(concurrent_uploading_sst_count) = self.concurrent_uploading_sst_count + && self.concurrent_upload_join_handle.len() >= concurrent_uploading_sst_count + { + self.concurrent_upload_join_handle + .next() + .verbose_instrument_await("upload") + .await + .unwrap() + .map_err(HummockError::sstable_upload_error)??; + } } Ok(()) } /// Finalizes all the tables to be ids, blocks and metadata. - pub async fn finish(mut self) -> HummockResult> { + pub async fn finish(mut self) -> HummockResult> { + use futures::future::try_join_all; self.seal_current().await?; + try_join_all(self.concurrent_upload_join_handle.into_iter()) + .await + .map_err(HummockError::sstable_upload_error)?; Ok(self.sst_outputs) } } @@ -504,6 +527,7 @@ mod tests { Arc::new(CompactorMetrics::unused()), None, table_partition_vnode, + None, ); let mut table_key = VirtualNode::from_index(0).to_be_bytes().to_vec(); diff --git a/src/storage/src/opts.rs b/src/storage/src/opts.rs index e9792c5d5e4db..143b6bba37981 100644 --- a/src/storage/src/opts.rs +++ b/src/storage/src/opts.rs @@ -128,6 +128,8 @@ pub struct StorageOpts { pub mem_table_spill_threshold: usize, + pub compactor_concurrent_uploading_sst_count: Option, + pub object_store_config: ObjectStoreConfig, } @@ -218,6 +220,9 @@ impl From<(&RwConfig, &SystemParamsReader, &StorageMemoryConfig)> for StorageOpt .compactor_fast_max_compact_delete_ratio, compactor_fast_max_compact_task_size: c.storage.compactor_fast_max_compact_task_size, compactor_iter_max_io_retry_times: c.storage.compactor_iter_max_io_retry_times, + compactor_concurrent_uploading_sst_count: c + .storage + .compactor_concurrent_uploading_sst_count, } } } From ef6d6c6f48429bb410265acfffdbb42676ae9c69 Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 13 Jun 2024 14:38:20 +0800 Subject: [PATCH 32/37] feat(source): support JSON schema addtionalProperties (map) (#17110) Signed-off-by: xxchan --- e2e_test/source/basic/nosim_kafka.slt | 20 ++++++++++++++-- scripts/source/test_data/kafka_json_schema.1 | 4 ++-- src/connector/src/parser/json_parser.rs | 21 +++++++++++++---- src/frontend/src/handler/create_source.rs | 24 ++++++++++++-------- 4 files changed, 50 insertions(+), 19 deletions(-) diff --git a/e2e_test/source/basic/nosim_kafka.slt b/e2e_test/source/basic/nosim_kafka.slt index 0cef4889c3fe7..b773126c9a7c1 100644 --- a/e2e_test/source/basic/nosim_kafka.slt +++ b/e2e_test/source/basic/nosim_kafka.slt @@ -39,6 +39,22 @@ CREATE TABLE kafka_json_schema_plain with ( kafka.scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE JSON (schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}'); + +query +describe kafka_json_schema_plain; +---- +dimensions (empty) false NULL +map jsonb false NULL +notMap (empty) false NULL +price double precision false NULL +productId bigint false NULL +productName character varying false NULL +tags character varying[] false NULL +_row_id serial true NULL +primary key _row_id NULL NULL +distribution key _row_id NULL NULL +table description kafka_json_schema_plain NULL NULL + statement ok CREATE TABLE kafka_json_schema_upsert (PRIMARY KEY(rw_key)) INCLUDE KEY AS rw_key @@ -83,10 +99,10 @@ select count(*) from debezium_compact; query TFITT select - "dimensions", "price", "productId", "productName", "tags" + * from kafka_json_schema_plain ---- -(9.5,7,12) 12.5 1 An ice sculpture {cold,ice} +(9.5,7,12) {"foo": "bar"} (b) 12.5 1 An ice sculpture {cold,ice} query TFITT select diff --git a/scripts/source/test_data/kafka_json_schema.1 b/scripts/source/test_data/kafka_json_schema.1 index e71485885a654..afa96107864cb 100644 --- a/scripts/source/test_data/kafka_json_schema.1 +++ b/scripts/source/test_data/kafka_json_schema.1 @@ -1,2 +1,2 @@ -{"$schema":"https://json-schema.org/draft/2020-12/schema","$id":"https://example.com/product.schema.json","title":"Product","description":"A product from Acme's catalog","type":"object","properties":{"productId":{"description":"The unique identifier for a product","type":"integer"},"productName":{"description":"Name of the product","type":"string"},"price":{"description":"The price of the product","type":"number","exclusiveMinimum":0},"tags":{"description":"Tags for the product","type":"array","items":{"type":"string"},"minItems":1,"uniqueItems":true},"dimensions":{"type":"object","properties":{"length":{"type":"number"},"width":{"type":"number"},"height":{"type":"number"}},"required":["length","width","height"]}},"required":["productId","productName","price"]} -{"productId":1,"productName":"An ice sculpture","price":12.5,"tags":["cold","ice"],"dimensions":{"length":7,"width":12,"height":9.5}} \ No newline at end of file +{"$schema":"https://json-schema.org/draft/2020-12/schema","$id":"https://example.com/product.schema.json","title":"Product","description":"A product from Acme's catalog","type":"object","properties":{"productId":{"description":"The unique identifier for a product","type":"integer"},"productName":{"description":"Name of the product","type":"string"},"price":{"description":"The price of the product","type":"number","exclusiveMinimum":0},"tags":{"description":"Tags for the product","type":"array","items":{"type":"string"},"minItems":1,"uniqueItems":true},"dimensions":{"type":"object","properties":{"length":{"type":"number"},"width":{"type":"number"},"height":{"type":"number"}},"required":["length","width","height"]},"map":{"type":"object","additionalProperties":{"type":"string"}},"notMap":{"type":"object","additionalProperties":{"type":"string"},"properties":{"a":{"type":"string"}}}},"required":["productId","productName","price"]} +{"productId":1,"productName":"An ice sculpture","price":12.5,"tags":["cold","ice"],"dimensions":{"length":7,"width":12,"height":9.5},"map":{"foo":"bar"},"notMap":{"a":"b","ignored":"c"}} \ No newline at end of file diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 9d62d76eff7a7..89f118eb1022f 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -26,6 +26,7 @@ use std::collections::BTreeMap; use anyhow::Context as _; use apache_avro::Schema; use jst::{convert_avro, Context}; +use risingwave_connector_codec::decoder::avro::MapHandling; use risingwave_pb::plan_common::ColumnDesc; use super::util::{bytes_from_url, get_kafka_topic}; @@ -80,7 +81,7 @@ impl JsonAccessBuilder { } } -pub async fn schema_to_columns( +pub async fn fetch_json_schema_and_map_to_columns( schema_location: &str, schema_registry_auth: Option, props: &BTreeMap, @@ -98,11 +99,21 @@ pub async fn schema_to_columns( let bytes = bytes_from_url(url, None).await?; serde_json::from_slice(&bytes)? }; - let context = Context::default(); - let avro_schema = convert_avro(&json_schema, context).to_string(); + json_schema_to_columns(&json_schema) +} + +/// FIXME: when the JSON schema is invalid, it will panic. +/// +/// ## Notes on type conversion +/// Map will be used when an object doesn't have `properties` but has `additionalProperties`. +/// When an object has `properties` and `additionalProperties`, the latter will be ignored. +/// +/// +/// TODO: examine other stuff like `oneOf`, `patternProperties`, etc. +fn json_schema_to_columns(json_schema: &serde_json::Value) -> ConnectorResult> { + let avro_schema = convert_avro(json_schema, Context::default()).to_string(); let schema = Schema::parse_str(&avro_schema).context("failed to parse avro schema")?; - // TODO: do we need to support map type here? - avro_schema_to_column_descs(&schema, None).map_err(Into::into) + avro_schema_to_column_descs(&schema, Some(MapHandling::Jsonb)).map_err(Into::into) } #[cfg(test)] diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index f10f951665208..a29aa86907e0f 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -32,8 +32,8 @@ use risingwave_connector::parser::additional_columns::{ build_additional_column_catalog, get_supported_additional_columns, }; use risingwave_connector::parser::{ - schema_to_columns, AvroParserConfig, DebeziumAvroParserConfig, ProtobufParserConfig, - SpecificParserConfig, TimestamptzHandling, DEBEZIUM_IGNORE_KEY, + fetch_json_schema_and_map_to_columns, AvroParserConfig, DebeziumAvroParserConfig, + ProtobufParserConfig, SpecificParserConfig, TimestamptzHandling, DEBEZIUM_IGNORE_KEY, }; use risingwave_connector::schema::schema_registry::{ name_strategy_from_str, SchemaRegistryAuth, SCHEMA_REGISTRY_PASSWORD, SCHEMA_REGISTRY_USERNAME, @@ -102,14 +102,18 @@ async fn extract_json_table_schema( auth }); Ok(Some( - schema_to_columns(&schema_location.0, schema_registry_auth, with_properties) - .await? - .into_iter() - .map(|col| ColumnCatalog { - column_desc: col.into(), - is_hidden: false, - }) - .collect_vec(), + fetch_json_schema_and_map_to_columns( + &schema_location.0, + schema_registry_auth, + with_properties, + ) + .await? + .into_iter() + .map(|col| ColumnCatalog { + column_desc: col.into(), + is_hidden: false, + }) + .collect_vec(), )) } } From a59356f6f8ea3c75bad79b05cf7c39240cb79753 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 13 Jun 2024 15:00:51 +0800 Subject: [PATCH 33/37] feat(catalog): add `rw_catalog.actor_id_to_ddl`, `rw_catalog.fragment_id_to_ddl` (#17229) --- .../catalog/system_catalog/rw_catalog/mod.rs | 3 ++ .../rw_catalog/rw_actor_id_to_ddl.rs | 42 +++++++++++++++++++ .../rw_catalog/rw_fragment_id_to_ddl.rs | 40 ++++++++++++++++++ 3 files changed, 85 insertions(+) create mode 100644 src/frontend/src/catalog/system_catalog/rw_catalog/rw_actor_id_to_ddl.rs create mode 100644 src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragment_id_to_ddl.rs diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs index 502c41561b65f..4e0e165a6d524 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs @@ -56,3 +56,6 @@ mod rw_user_secrets; mod rw_users; mod rw_views; mod rw_worker_nodes; + +mod rw_actor_id_to_ddl; +mod rw_fragment_id_to_ddl; diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actor_id_to_ddl.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actor_id_to_ddl.rs new file mode 100644 index 0000000000000..95e269f0d5d5c --- /dev/null +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actor_id_to_ddl.rs @@ -0,0 +1,42 @@ +// 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::types::Fields; +use risingwave_frontend_macro::system_catalog; + +/// Provides a mapping from `actor_id` to its ddl info. +#[system_catalog( +view, +"rw_catalog.rw_actor_id_to_ddl", +"with + actor_to_job_id as (select actor_id, a.fragment_id, table_id from rw_fragments f join rw_actors a on f.fragment_id = a.fragment_id), + job_id_to_mv as (select actor_id, fragment_id, d.id as job_id, schema_id, 'mv' as ddl_type, name from rw_materialized_views d join actor_to_job_id a on d.id = a.table_id), + job_id_to_sink as (select actor_id, fragment_id, d.id as job_id, schema_id, 'sink' as ddl_type, name from rw_sinks d join actor_to_job_id a on d.id = a.table_id), + job_id_to_source as (select actor_id, fragment_id, d.id as job_id, schema_id, 'source' as ddl_type, name from rw_sources d join actor_to_job_id a on d.id = a.table_id), + job_id_to_table as (select actor_id, fragment_id, d.id as job_id, schema_id, 'table' as ddl_type, name from rw_tables d join actor_to_job_id a on d.id = a.table_id) + select * from job_id_to_mv + union all select * from job_id_to_sink + union all select * from job_id_to_source + union all select * from job_id_to_table" +)] +#[derive(Fields)] +struct RwActorIdToDdl { + #[primary_key] + actor_id: i32, + fragment_id: i32, + job_id: i32, + schema_id: i32, + ddl_type: String, + name: String, +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragment_id_to_ddl.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragment_id_to_ddl.rs new file mode 100644 index 0000000000000..094e85903a31a --- /dev/null +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragment_id_to_ddl.rs @@ -0,0 +1,40 @@ +// 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::types::Fields; +use risingwave_frontend_macro::system_catalog; + +/// Provides a mapping from `actor_id` to its ddl info. +#[system_catalog( +view, +"rw_catalog.rw_fragment_id_to_ddl", +"with + job_id_to_mv as (select fragment_id, d.id as job_id, schema_id, 'mv' as ddl_type, name from rw_materialized_views d join rw_fragments f on d.id = f.table_id), + job_id_to_sink as (select fragment_id, d.id as job_id, schema_id, 'sink' as ddl_type, name from rw_sinks d join rw_fragments f on d.id = f.table_id), + job_id_to_source as (select fragment_id, d.id as job_id, schema_id, 'source' as ddl_type, name from rw_sources d join rw_fragments f on d.id = f.table_id), + job_id_to_table as (select fragment_id, d.id as job_id, schema_id, 'table' as ddl_type, name from rw_tables d join rw_fragments f on d.id = f.table_id) + select * from job_id_to_mv + union all select * from job_id_to_sink + union all select * from job_id_to_source + union all select * from job_id_to_table" +)] +#[derive(Fields)] +struct RwFragmentIdToDdl { + #[primary_key] + fragment_id: i32, + job_id: i32, + schema_id: i32, + ddl_type: String, + name: String, +} From 759b2228fa4f556d281814afc4842cdd8ba9c23c Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 13 Jun 2024 15:15:48 +0800 Subject: [PATCH 34/37] refactor(storage): refactor compaction group manager txn (#17070) --- src/meta/node/src/server.rs | 2 +- src/meta/src/controller/session_params.rs | 2 +- src/meta/src/hummock/compactor_manager.rs | 2 +- src/meta/src/hummock/manager/compaction.rs | 16 +- .../manager/compaction_group_manager.rs | 486 ++++++++---------- src/meta/src/hummock/manager/context.rs | 4 +- src/meta/src/hummock/manager/mod.rs | 14 +- src/meta/src/hummock/manager/tests.rs | 4 +- src/meta/src/hummock/manager/timer_task.rs | 9 +- src/meta/src/manager/cluster.rs | 2 +- src/meta/src/manager/env.rs | 4 +- src/meta/src/model/catalog.rs | 2 +- src/storage/hummock_sdk/src/version.rs | 4 +- 13 files changed, 255 insertions(+), 296 deletions(-) diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index 8802ebb37951e..e8b738305dce7 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -659,7 +659,7 @@ pub async fn start_service_as_election_leader( ); let health_srv = HealthServiceImpl::new(); let backup_srv = BackupServiceImpl::new(backup_manager); - let telemetry_srv = TelemetryInfoServiceImpl::new(env.meta_store_ref()); + let telemetry_srv = TelemetryInfoServiceImpl::new(env.meta_store()); let system_params_srv = SystemParamsServiceImpl::new(env.system_params_manager_impl_ref()); let session_params_srv = SessionParamsServiceImpl::new(env.session_params_manager_impl_ref()); let serving_srv = diff --git a/src/meta/src/controller/session_params.rs b/src/meta/src/controller/session_params.rs index 4a27967fa2b0a..566170a0ef4d2 100644 --- a/src/meta/src/controller/session_params.rs +++ b/src/meta/src/controller/session_params.rs @@ -151,7 +151,7 @@ mod tests { use sea_orm::QueryFilter; let env = MetaSrvEnv::for_test_with_sql_meta_store().await; - let meta_store = env.meta_store().as_sql(); + let meta_store = env.meta_store_ref().as_sql(); let init_params = SessionConfig::default(); // init system parameter controller as first launch. diff --git a/src/meta/src/hummock/compactor_manager.rs b/src/meta/src/hummock/compactor_manager.rs index 0876f31d211e3..ab0c868f703c4 100644 --- a/src/meta/src/hummock/compactor_manager.rs +++ b/src/meta/src/hummock/compactor_manager.rs @@ -138,7 +138,7 @@ impl CompactorManagerInner { use risingwave_meta_model_v2::compaction_task; use sea_orm::EntityTrait; // Retrieve the existing task assignments from metastore. - let task_assignment: Vec = match env.meta_store() { + let task_assignment: Vec = match env.meta_store_ref() { MetaStoreImpl::Kv(meta_store) => CompactTaskAssignment::list(meta_store).await?, MetaStoreImpl::Sql(sql_meta_store) => compaction_task::Entity::find() .all(&sql_meta_store.conn) diff --git a/src/meta/src/hummock/manager/compaction.rs b/src/meta/src/hummock/manager/compaction.rs index 7a27061c8d373..1969fc8ffc348 100644 --- a/src/meta/src/hummock/manager/compaction.rs +++ b/src/meta/src/hummock/manager/compaction.rs @@ -704,15 +704,15 @@ impl HummockManager { // When the last table of a compaction group is deleted, the compaction group (and its // config) is destroyed as well. Then a compaction task for this group may come later and // cannot find its config. - let group_config = match self - .compaction_group_manager - .read() - .await - .try_get_compaction_group_config(compaction_group_id) - { - Some(config) => config, - None => continue, + let group_config = { + let config_manager = self.compaction_group_manager.read().await; + + match config_manager.try_get_compaction_group_config(compaction_group_id) { + Some(config) => config, + None => continue, + } }; + // StoredIdGenerator already implements ids pre-allocation by ID_PREALLOCATE_INTERVAL. let task_id = next_compaction_task_id(&self.env).await?; diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index 063ce4e0a58b1..3a3d596844e95 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -42,17 +42,15 @@ use crate::hummock::compaction::compaction_config::{ }; use crate::hummock::error::{Error, Result}; use crate::hummock::manager::transaction::HummockVersionTransaction; -use crate::hummock::manager::versioning::{calc_new_write_limits, Versioning}; +use crate::hummock::manager::versioning::Versioning; use crate::hummock::manager::{commit_multi_var, HummockManager}; -use crate::hummock::metrics_utils::{ - remove_compaction_group_in_sst_stat, trigger_write_stop_stats, -}; +use crate::hummock::metrics_utils::remove_compaction_group_in_sst_stat; use crate::hummock::model::CompactionGroup; use crate::hummock::sequence::{next_compaction_group_id, next_sstable_object_id}; use crate::manager::{MetaSrvEnv, MetaStoreImpl}; -use crate::model::{ - BTreeMapEntryTransaction, BTreeMapTransaction, MetadataModel, MetadataModelError, -}; +use crate::model::{BTreeMapTransaction, MetadataModel, MetadataModelError}; + +type CompactionGroupTransaction<'a> = BTreeMapTransaction<'a, CompactionGroupId, CompactionGroup>; impl CompactionGroupManager { pub(super) async fn new(env: &MetaSrvEnv) -> Result { @@ -69,13 +67,38 @@ impl CompactionGroupManager { ) -> Result { let mut compaction_group_manager = CompactionGroupManager { compaction_groups: BTreeMap::new(), - default_config, + default_config: Arc::new(default_config), write_limit: Default::default(), - meta_store_impl: env.meta_store_ref(), }; - compaction_group_manager.init().await?; + + let loaded_compaction_groups: BTreeMap = + match env.meta_store_ref() { + MetaStoreImpl::Kv(meta_store) => CompactionGroup::list(meta_store) + .await? + .into_iter() + .map(|cg| (cg.group_id(), cg)) + .collect(), + MetaStoreImpl::Sql(sql_meta_store) => { + use sea_orm::EntityTrait; + compaction_config::Entity::find() + .all(&sql_meta_store.conn) + .await + .map_err(MetadataModelError::from)? + .into_iter() + .map(|m| (m.compaction_group_id as CompactionGroupId, m.into())) + .collect() + } + }; + + compaction_group_manager.init(loaded_compaction_groups); Ok(compaction_group_manager) } + + fn init(&mut self, loaded_compaction_groups: BTreeMap) { + if !loaded_compaction_groups.is_empty() { + self.compaction_groups = loaded_compaction_groups; + } + } } impl HummockManager { @@ -167,7 +190,10 @@ impl HummockManager { } let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); + let mut compaction_group_manager = self.compaction_group_manager.write().await; let current_version = &versioning.current_version; + let default_config = compaction_group_manager.default_compaction_config(); + let mut compaction_groups_txn = compaction_group_manager.start_compaction_groups_txn(); for (table_id, _) in pairs { if let Some(old_group) = @@ -210,15 +236,17 @@ impl HummockManager { .entry(group_id) .or_default() .group_deltas; - let config = self - .compaction_group_manager - .write() - .await - .get_or_insert_compaction_group_config(group_id) - .await? - .compaction_config - .as_ref() - .clone(); + + let config = + match compaction_groups_txn.try_get_compaction_group_config(group_id) { + Some(config) => config.compaction_config.as_ref().clone(), + None => { + compaction_groups_txn + .create_compaction_groups(group_id, default_config.clone()); + default_config.as_ref().clone() + } + }; + group_deltas.push(GroupDelta { delta_type: Some(DeltaType::GroupConstruct(GroupConstruct { group_config: Some(config), @@ -233,6 +261,7 @@ impl HummockManager { .entry(group_id) .or_default() .group_deltas; + group_deltas.push(GroupDelta { delta_type: Some(DeltaType::GroupMetaChange(GroupMetaChange { table_ids_add: vec![*table_id], @@ -251,7 +280,7 @@ impl HummockManager { .is_none()); } new_version_delta.pre_apply(); - commit_multi_var!(self.meta_store_ref(), version)?; + commit_multi_var!(self.meta_store_ref(), version, compaction_groups_txn)?; Ok(()) } @@ -334,22 +363,23 @@ impl HummockManager { delta_type: Some(DeltaType::GroupDestroy(GroupDestroy {})), }); } - new_version_delta.pre_apply(); - commit_multi_var!(self.meta_store_ref(), version)?; for (group_id, max_level) in groups_to_remove { remove_compaction_group_in_sst_stat(&self.metrics, group_id, max_level); } + new_version_delta.pre_apply(); + // Purge may cause write to meta store. If it hurts performance while holding versioning // lock, consider to make it in batch. - self.compaction_group_manager - .write() - .await - .purge(HashSet::from_iter(get_compaction_group_ids( - &versioning.current_version, - ))) - .await?; + let mut compaction_group_manager = self.compaction_group_manager.write().await; + let mut compaction_groups_txn = compaction_group_manager.start_compaction_groups_txn(); + + compaction_groups_txn.purge(HashSet::from_iter(get_compaction_group_ids( + version.latest_version(), + ))); + commit_multi_var!(self.meta_store_ref(), version, compaction_groups_txn)?; + Ok(()) } @@ -357,21 +387,25 @@ impl HummockManager { &self, compaction_group_ids: &[CompactionGroupId], config_to_update: &[MutableConfig], - ) -> Result> { - let result = self - .compaction_group_manager - .write() - .await - .update_compaction_config(compaction_group_ids, config_to_update) - .await?; + ) -> Result<()> { + { + // Avoid lock conflicts with `try_update_write_limits`` + let mut compaction_group_manager = self.compaction_group_manager.write().await; + let mut compaction_groups_txn = compaction_group_manager.start_compaction_groups_txn(); + compaction_groups_txn + .update_compaction_config(compaction_group_ids, config_to_update)?; + commit_multi_var!(self.meta_store_ref(), compaction_groups_txn)?; + } + if config_to_update .iter() .any(|c| matches!(c, MutableConfig::Level0StopWriteThresholdSubLevelNumber(_))) { + // Update write limits with lock self.try_update_write_limits(compaction_group_ids).await; } - Ok(result) + Ok(()) } /// Gets complete compaction group info. @@ -380,24 +414,25 @@ impl HummockManager { let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); let current_version = &versioning.current_version; - let mut compaction_groups = vec![]; + let mut results = vec![]; + let compaction_group_manager = self.compaction_group_manager.read().await; + for levels in current_version.levels.values() { - let config = self - .compaction_group_manager - .read() - .await + let compaction_config = compaction_group_manager .try_get_compaction_group_config(levels.group_id) .unwrap() - .compaction_config; + .compaction_config + .as_ref() + .clone(); let group = CompactionGroupInfo { id: levels.group_id, parent_id: levels.parent_group_id, member_table_ids: levels.member_table_ids.clone(), - compaction_config: Some(config.as_ref().clone()), + compaction_config: Some(compaction_config), }; - compaction_groups.push(group); + results.push(group); } - compaction_groups + results } /// Splits a compaction group into two. The new one will contain `table_ids`. @@ -415,21 +450,19 @@ impl HummockManager { ) .await?; - Ok(result.0) + Ok(result) } /// move some table to another compaction-group. Create a new compaction group if it does not /// exist. - /// TODO: Move `table_to_partition` in result to compaction group pub async fn move_state_table_to_compaction_group( &self, parent_group_id: CompactionGroupId, table_ids: &[StateTableId], partition_vnode_count: u32, - ) -> Result<(CompactionGroupId, BTreeMap)> { - let mut table_to_partition = BTreeMap::default(); + ) -> Result { if table_ids.is_empty() { - return Ok((parent_group_id, table_to_partition)); + return Ok(parent_group_id); } let table_ids = table_ids.iter().cloned().unique().collect_vec(); let compaction_guard = self.compaction.write().await; @@ -483,7 +516,9 @@ impl HummockManager { .compaction_group_manager .read() .await - .default_compaction_config(); + .default_compaction_config() + .as_ref() + .clone(); config.split_weight_by_vnode = partition_vnode_count; new_version_delta.group_deltas.insert( @@ -520,20 +555,12 @@ impl HummockManager { let (new_compaction_group_id, config) = new_group; { let mut compaction_group_manager = self.compaction_group_manager.write().await; - let insert = BTreeMapEntryTransaction::new_insert( - &mut compaction_group_manager.compaction_groups, - new_compaction_group_id, - CompactionGroup { - group_id: new_compaction_group_id, - compaction_config: Arc::new(config), - }, - ); + let mut compaction_groups_txn = compaction_group_manager.start_compaction_groups_txn(); + compaction_groups_txn + .create_compaction_groups(new_compaction_group_id, Arc::new(config)); + new_version_delta.pre_apply(); - commit_multi_var!(self.meta_store_ref(), version, insert)?; - // Currently, only splitting out a single table_id is supported. - for table_id in table_ids { - table_to_partition.insert(table_id, partition_vnode_count); - } + commit_multi_var!(self.meta_store_ref(), version, compaction_groups_txn)?; } let mut canceled_tasks = vec![]; @@ -571,7 +598,7 @@ impl HummockManager { .with_label_values(&[&parent_group_id.to_string()]) .inc(); - Ok((target_compaction_group_id, table_to_partition)) + Ok(target_compaction_group_id) } pub async fn calculate_compaction_group_statistic(&self) -> Vec { @@ -614,71 +641,11 @@ impl HummockManager { ) -> Result<()> { // 1. Due to version compatibility, we fix some of the configuration of older versions after hummock starts. let current_version = &versioning_guard.current_version; - let all_group_ids = get_compaction_group_ids(current_version); - let mut configs = compaction_group_manager - .get_or_insert_compaction_group_configs(&all_group_ids.collect_vec()) - .await?; - - // We've already lowered the default limit for write limit in PR-12183, and to prevent older clusters from continuing to use the outdated configuration, we've introduced a new logic to rewrite it in a uniform way. - let mut rewrite_cg_ids = vec![]; - let mut restore_cg_to_partition_vnode: HashMap> = - HashMap::default(); - for (cg_id, compaction_group_config) in &mut configs { - // update write limit - let relaxed_default_write_stop_level_count = 1000; - if compaction_group_config - .compaction_config - .level0_sub_level_compact_level_count - == relaxed_default_write_stop_level_count - { - rewrite_cg_ids.push(*cg_id); - } - - if let Some(levels) = current_version.levels.get(cg_id) { - if levels.member_table_ids.len() == 1 { - restore_cg_to_partition_vnode.insert( - *cg_id, - vec![( - levels.member_table_ids[0], - compaction_group_config - .compaction_config - .split_weight_by_vnode, - )] - .into_iter() - .collect(), - ); - } - } - } - - if !rewrite_cg_ids.is_empty() { - tracing::info!("Compaction group {:?} configs rewrite ", rewrite_cg_ids); - - // update meta store - let result = compaction_group_manager - .update_compaction_config( - &rewrite_cg_ids, - &[ - MutableConfig::Level0StopWriteThresholdSubLevelNumber( - risingwave_common::config::default::compaction_config::level0_stop_write_threshold_sub_level_number(), - ), - ], - ) - .await?; - - // update memory - for new_config in result { - configs.insert(new_config.group_id(), new_config); - } - } - - compaction_group_manager.write_limit = - calc_new_write_limits(configs, HashMap::new(), &versioning_guard.current_version); - trigger_write_stop_stats(&self.metrics, &compaction_group_manager.write_limit); - tracing::debug!( - "Hummock stopped write: {:#?}", - compaction_group_manager.write_limit - ); + let all_group_ids = get_compaction_group_ids(current_version).collect_vec(); + let default_config = compaction_group_manager.default_compaction_config(); + let mut compaction_groups_txn = compaction_group_manager.start_compaction_groups_txn(); + compaction_groups_txn.try_create_compaction_groups(&all_group_ids, default_config); + commit_multi_var!(self.meta_store_ref(), compaction_groups_txn)?; Ok(()) } @@ -693,91 +660,15 @@ impl HummockManager { /// 3. move existent table to new compaction group. pub(super) struct CompactionGroupManager { compaction_groups: BTreeMap, - default_config: CompactionConfig, + default_config: Arc, /// Tables that write limit is trigger for. pub write_limit: HashMap, - meta_store_impl: MetaStoreImpl, -} - -// init method -impl CompactionGroupManager { - async fn init(&mut self) -> Result<()> { - let loaded_compaction_groups: BTreeMap = - match &self.meta_store_impl { - MetaStoreImpl::Kv(meta_store) => CompactionGroup::list(meta_store) - .await? - .into_iter() - .map(|cg| (cg.group_id(), cg)) - .collect(), - MetaStoreImpl::Sql(sql_meta_store) => { - use sea_orm::EntityTrait; - compaction_config::Entity::find() - .all(&sql_meta_store.conn) - .await - .map_err(MetadataModelError::from)? - .into_iter() - .map(|m| (m.compaction_group_id as CompactionGroupId, m.into())) - .collect() - } - }; - if !loaded_compaction_groups.is_empty() { - self.compaction_groups = loaded_compaction_groups; - } - Ok(()) - } - - /// Initializes the config for a group. - /// Should only be used by compaction test. - pub(super) async fn init_compaction_config_for_replay( - &mut self, - group_id: CompactionGroupId, - config: CompactionConfig, - ) -> Result<()> { - let insert = BTreeMapEntryTransaction::new_insert( - &mut self.compaction_groups, - group_id, - CompactionGroup { - group_id, - compaction_config: Arc::new(config), - }, - ); - commit_multi_var!(self.meta_store_impl, insert)?; - Ok(()) - } } impl CompactionGroupManager { - /// Gets compaction group config for `compaction_group_id`, inserts default one if missing. - async fn get_or_insert_compaction_group_config( - &mut self, - compaction_group_id: CompactionGroupId, - ) -> Result { - let r = self - .get_or_insert_compaction_group_configs(&[compaction_group_id]) - .await?; - Ok(r.into_values().next().unwrap()) - } - - /// Gets compaction group configs for `compaction_group_ids`, inserts default one if missing. - async fn get_or_insert_compaction_group_configs( - &mut self, - compaction_group_ids: &[CompactionGroupId], - ) -> Result> { - let mut compaction_groups = BTreeMapTransaction::new(&mut self.compaction_groups); - for id in compaction_group_ids { - if compaction_groups.contains_key(id) { - continue; - } - let new_entry = CompactionGroup::new(*id, self.default_config.clone()); - compaction_groups.insert(*id, new_entry); - } - commit_multi_var!(self.meta_store_impl, compaction_groups)?; - - let r = compaction_group_ids - .iter() - .map(|id| (*id, self.compaction_groups[id].clone())) - .collect(); - Ok(r) + /// Starts a transaction to update compaction group configs. + pub fn start_compaction_groups_txn(&mut self) -> CompactionGroupTransaction<'_> { + CompactionGroupTransaction::new(&mut self.compaction_groups) } /// Tries to get compaction group config for `compaction_group_id`. @@ -788,53 +679,10 @@ impl CompactionGroupManager { self.compaction_groups.get(&compaction_group_id).cloned() } - pub(super) fn default_compaction_config(&self) -> CompactionConfig { + /// Tries to get compaction group config for `compaction_group_id`. + pub(super) fn default_compaction_config(&self) -> Arc { self.default_config.clone() } - - pub(super) async fn update_compaction_config( - &mut self, - compaction_group_ids: &[CompactionGroupId], - config_to_update: &[MutableConfig], - ) -> Result> { - let mut compaction_groups = BTreeMapTransaction::new(&mut self.compaction_groups); - let mut result = Vec::with_capacity(compaction_group_ids.len()); - for compaction_group_id in compaction_group_ids.iter().unique() { - let group = compaction_groups.get(compaction_group_id).ok_or_else(|| { - Error::CompactionGroup(format!("invalid group {}", *compaction_group_id)) - })?; - let mut config = group.compaction_config.as_ref().clone(); - update_compaction_config(&mut config, config_to_update); - if let Err(reason) = validate_compaction_config(&config) { - return Err(Error::CompactionGroup(reason)); - } - let mut new_group = group.clone(); - new_group.compaction_config = Arc::new(config); - compaction_groups.insert(*compaction_group_id, new_group.clone()); - result.push(new_group); - } - commit_multi_var!(self.meta_store_impl, compaction_groups)?; - Ok(result) - } - - /// Removes stale group configs. - async fn purge(&mut self, existing_groups: HashSet) -> Result<()> { - let mut compaction_groups = BTreeMapTransaction::new(&mut self.compaction_groups); - let stale_group = compaction_groups - .tree_ref() - .keys() - .cloned() - .filter(|k| !existing_groups.contains(k)) - .collect_vec(); - if stale_group.is_empty() { - return Ok(()); - } - for group in stale_group { - compaction_groups.remove(group); - } - commit_multi_var!(self.meta_store_impl, compaction_groups)?; - Ok(()) - } } fn update_compaction_config(target: &mut CompactionConfig, items: &[MutableConfig]) { @@ -896,6 +744,84 @@ fn update_compaction_config(target: &mut CompactionConfig, items: &[MutableConfi } } +impl<'a> CompactionGroupTransaction<'a> { + /// Inserts compaction group configs if they do not exist. + pub fn try_create_compaction_groups( + &mut self, + compaction_group_ids: &[CompactionGroupId], + config: Arc, + ) -> bool { + let mut trivial = true; + for id in compaction_group_ids { + if self.contains_key(id) { + continue; + } + let new_entry = CompactionGroup::new(*id, config.as_ref().clone()); + self.insert(*id, new_entry); + + trivial = false; + } + + !trivial + } + + pub fn create_compaction_groups( + &mut self, + compaction_group_id: CompactionGroupId, + config: Arc, + ) { + self.try_create_compaction_groups(&[compaction_group_id], config); + } + + /// Tries to get compaction group config for `compaction_group_id`. + pub(super) fn try_get_compaction_group_config( + &self, + compaction_group_id: CompactionGroupId, + ) -> Option<&CompactionGroup> { + self.get(&compaction_group_id) + } + + /// Removes stale group configs. + fn purge(&mut self, existing_groups: HashSet) { + let stale_group = self + .tree_ref() + .keys() + .cloned() + .filter(|k| !existing_groups.contains(k)) + .collect_vec(); + if stale_group.is_empty() { + return; + } + for group in stale_group { + self.remove(group); + } + } + + pub(super) fn update_compaction_config( + &mut self, + compaction_group_ids: &[CompactionGroupId], + config_to_update: &[MutableConfig], + ) -> Result> { + let mut results = HashMap::default(); + for compaction_group_id in compaction_group_ids.iter().unique() { + let group = self.get(compaction_group_id).ok_or_else(|| { + Error::CompactionGroup(format!("invalid group {}", *compaction_group_id)) + })?; + let mut config = group.compaction_config.as_ref().clone(); + update_compaction_config(&mut config, config_to_update); + if let Err(reason) = validate_compaction_config(&config) { + return Err(Error::CompactionGroup(reason)); + } + let mut new_group = group.clone(); + new_group.compaction_config = Arc::new(config); + self.insert(*compaction_group_id, new_group.clone()); + results.insert(new_group.group_id(), new_group); + } + + Ok(results) + } +} + #[cfg(test)] mod tests { use std::collections::BTreeMap; @@ -905,8 +831,11 @@ mod tests { use risingwave_pb::hummock::rise_ctl_update_compaction_config_request::mutable_config::MutableConfig; use risingwave_pb::meta::table_fragments::Fragment; + use crate::hummock::commit_multi_var; + use crate::hummock::error::Result; use crate::hummock::manager::compaction_group_manager::CompactionGroupManager; use crate::hummock::test_utils::setup_compute_env; + use crate::manager::MetaStoreImpl; use crate::model::TableFragments; #[tokio::test] @@ -914,21 +843,46 @@ mod tests { let (env, ..) = setup_compute_env(8080).await; let mut inner = CompactionGroupManager::new(&env).await.unwrap(); assert_eq!(inner.compaction_groups.len(), 2); - inner - .update_compaction_config(&[100, 200], &[]) + + async fn update_compaction_config( + meta: &MetaStoreImpl, + inner: &mut CompactionGroupManager, + cg_ids: &[u64], + config_to_update: &[MutableConfig], + ) -> Result<()> { + let mut compaction_groups_txn = inner.start_compaction_groups_txn(); + compaction_groups_txn.update_compaction_config(cg_ids, config_to_update)?; + commit_multi_var!(meta, compaction_groups_txn) + } + + async fn insert_compaction_group_configs( + meta: &MetaStoreImpl, + inner: &mut CompactionGroupManager, + cg_ids: &[u64], + ) { + let default_config = inner.default_compaction_config(); + let mut compaction_groups_txn = inner.start_compaction_groups_txn(); + if compaction_groups_txn.try_create_compaction_groups(cg_ids, default_config) { + commit_multi_var!(meta, compaction_groups_txn).unwrap(); + } + } + + update_compaction_config(env.meta_store_ref(), &mut inner, &[100, 200], &[]) .await .unwrap_err(); - inner - .get_or_insert_compaction_group_configs(&[100, 200]) - .await - .unwrap(); + insert_compaction_group_configs(env.meta_store_ref(), &mut inner, &[100, 200]).await; assert_eq!(inner.compaction_groups.len(), 4); let mut inner = CompactionGroupManager::new(&env).await.unwrap(); assert_eq!(inner.compaction_groups.len(), 4); - inner - .update_compaction_config(&[100, 200], &[MutableConfig::MaxSubCompaction(123)]) - .await - .unwrap(); + + update_compaction_config( + env.meta_store_ref(), + &mut inner, + &[100, 200], + &[MutableConfig::MaxSubCompaction(123)], + ) + .await + .unwrap(); assert_eq!(inner.compaction_groups.len(), 4); assert_eq!( inner diff --git a/src/meta/src/hummock/manager/context.rs b/src/meta/src/hummock/manager/context.rs index 155dd362c0909..982a94fd5f9db 100644 --- a/src/meta/src/hummock/manager/context.rs +++ b/src/meta/src/hummock/manager/context.rs @@ -99,7 +99,7 @@ impl HummockManager { ) -> Result<()> { let mut context_info = self.context_info.write().await; context_info - .release_contexts(context_ids, self.meta_store_ref()) + .release_contexts(context_ids, self.env.meta_store()) .await?; #[cfg(test)] { @@ -188,7 +188,7 @@ impl HummockManager { } context_info - .release_contexts(&invalid_context_ids, self.meta_store_ref()) + .release_contexts(&invalid_context_ids, self.env.meta_store()) .await?; Ok(invalid_context_ids) diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 47209ddf1fff2..8a49d91a55fc3 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -297,7 +297,7 @@ impl HummockManager { Ok(instance) } - fn meta_store_ref(&self) -> MetaStoreImpl { + fn meta_store_ref(&self) -> &MetaStoreImpl { self.env.meta_store_ref() } @@ -495,22 +495,22 @@ impl HummockManager { ); } + let mut compaction_group_manager = self.compaction_group_manager.write().await; + let mut compaction_groups_txn = compaction_group_manager.start_compaction_groups_txn(); for group in &compaction_groups { let mut pairs = vec![]; for table_id in group.member_table_ids.clone() { pairs.push((table_id as StateTableId, group.id)); } let group_config = group.compaction_config.clone().unwrap(); - self.compaction_group_manager - .write() - .await - .init_compaction_config_for_replay(group.id, group_config) - .await - .unwrap(); + compaction_groups_txn.create_compaction_groups(group.id, Arc::new(group_config)); + self.register_table_ids_for_test(&pairs).await?; tracing::info!("Registered table ids {:?}", pairs); } + commit_multi_var!(self.meta_store_ref(), compaction_groups_txn)?; + // Notify that tables have created for table in table_catalogs { self.env diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 90ba6db61de4e..5b62e5f0694b7 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -100,7 +100,7 @@ fn get_compaction_group_object_ids( } async fn list_pinned_snapshot_from_meta_store(env: &MetaSrvEnv) -> Vec { - match env.meta_store() { + match env.meta_store_ref() { MetaStoreImpl::Kv(meta_store) => HummockPinnedSnapshot::list(meta_store).await.unwrap(), MetaStoreImpl::Sql(sql_meta_store) => { use risingwave_meta_model_v2::hummock_pinned_snapshot; @@ -117,7 +117,7 @@ async fn list_pinned_snapshot_from_meta_store(env: &MetaSrvEnv) -> Vec Vec { - match env.meta_store() { + match env.meta_store_ref() { MetaStoreImpl::Kv(meta_store) => HummockPinnedVersion::list(meta_store).await.unwrap(), MetaStoreImpl::Sql(sql_meta_store) => { use risingwave_meta_model_v2::hummock_pinned_version; diff --git a/src/meta/src/hummock/manager/timer_task.rs b/src/meta/src/hummock/manager/timer_task.rs index 6b68950a28e3b..b7c8cae4b260e 100644 --- a/src/meta/src/hummock/manager/timer_task.rs +++ b/src/meta/src/hummock/manager/timer_task.rs @@ -565,8 +565,13 @@ impl HummockManager { ) .await; match ret { - Ok((new_group_id, table_vnode_partition_count)) => { - tracing::info!("move state table [{}] from group-{} to group-{} success table_vnode_partition_count {:?}", table_id, parent_group_id, new_group_id, table_vnode_partition_count); + Ok(new_group_id) => { + tracing::info!( + "move state table [{}] from group-{} to group-{} success", + table_id, + parent_group_id, + new_group_id + ); } Err(e) => { tracing::info!( diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 4dd4a257c5b45..876050c36ae6c 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -568,7 +568,7 @@ impl ClusterManagerCore { pub const MAX_WORKER_REUSABLE_ID_COUNT: usize = 1 << Self::MAX_WORKER_REUSABLE_ID_BITS; async fn new(env: MetaSrvEnv) -> MetaResult { - let meta_store = env.meta_store().as_kv(); + let meta_store = env.meta_store_ref().as_kv(); let mut workers = Worker::list(meta_store).await?; let used_transactional_ids: HashSet<_> = workers diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 0b0c2ed2a3ced..b623e441c0c22 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -462,11 +462,11 @@ impl MetaSrvEnv { Ok(env) } - pub fn meta_store_ref(&self) -> MetaStoreImpl { + pub fn meta_store(&self) -> MetaStoreImpl { self.meta_store_impl.clone() } - pub fn meta_store(&self) -> &MetaStoreImpl { + pub fn meta_store_ref(&self) -> &MetaStoreImpl { &self.meta_store_impl } diff --git a/src/meta/src/model/catalog.rs b/src/meta/src/model/catalog.rs index c11be01d1a599..8d89080ae2462 100644 --- a/src/meta/src/model/catalog.rs +++ b/src/meta/src/model/catalog.rs @@ -96,7 +96,7 @@ mod tests { #[tokio::test] async fn test_database() -> MetadataModelResult<()> { let env = MetaSrvEnv::for_test().await; - let store = env.meta_store().as_kv(); + let store = env.meta_store_ref().as_kv(); let databases = Database::list(store).await?; assert!(databases.is_empty()); assert!(Database::select(store, &0).await.unwrap().is_none()); diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 51780b1bc7334..b30c1ba0e5191 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -266,7 +266,7 @@ impl HummockVersion { self.safe_epoch } - pub fn create_init_version(default_compaction_config: CompactionConfig) -> HummockVersion { + pub fn create_init_version(default_compaction_config: Arc) -> HummockVersion { let mut init_version = HummockVersion { id: FIRST_VERSION_ID, levels: Default::default(), @@ -282,7 +282,7 @@ impl HummockVersion { ] { init_version.levels.insert( group_id, - build_initial_compaction_group_levels(group_id, &default_compaction_config), + build_initial_compaction_group_levels(group_id, default_compaction_config.as_ref()), ); } init_version From c2b0ead320066e78af8aadfff94c3ccf40c42be2 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 13 Jun 2024 15:37:20 +0800 Subject: [PATCH 35/37] chore: bump mysql java library (#17234) --- java/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/java/pom.xml b/java/pom.xml index f1ee457ef3b84..5f0327bf8ffc9 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -83,7 +83,7 @@ 1.18.0 1.17.6 42.5.5 - 8.0.33 + 8.3.0 4.11.1 3.45.0.0 2.21.42 @@ -570,4 +570,4 @@ https://s01.oss.sonatype.org/service/local/staging/deploy/maven2/ - \ No newline at end of file + From 4d32c189e785b00c058d5ff89b7e728445a2af26 Mon Sep 17 00:00:00 2001 From: Tao Wu Date: Thu, 13 Jun 2024 15:45:39 +0800 Subject: [PATCH 36/37] chore: update tokio 0.2.24 to 0.2 (#17233) --- src/tests/e2e_extended_mode/Cargo.toml | 2 +- src/tests/simulation/Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/tests/e2e_extended_mode/Cargo.toml b/src/tests/e2e_extended_mode/Cargo.toml index ecbf751fe1004..cbc7831ea7a03 100644 --- a/src/tests/e2e_extended_mode/Cargo.toml +++ b/src/tests/e2e_extended_mode/Cargo.toml @@ -19,7 +19,7 @@ chrono = { version = "0.4", features = ['serde'] } clap = { workspace = true } pg_interval = "0.4" rust_decimal = { version = "1.35", features = ["db-postgres"] } -tokio = { version = "0.2.24", package = "madsim-tokio", features = [ +tokio = { version = "0.2", package = "madsim-tokio", features = [ "rt", "macros", "rt-multi-thread", diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index 57768643eb7dc..5fbfc0e19f6fd 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -53,7 +53,7 @@ serde_json = "1.0.107" sqllogictest = "0.20" tempfile = "3" tikv-jemallocator = { workspace = true } -tokio = { version = "0.2.24", package = "madsim-tokio" } +tokio = { version = "0.2", package = "madsim-tokio" } tokio-postgres = "0.7" tokio-stream = "0.1" tracing = "0.1" From 3b823d0175080be867751b486dc7c572dd51188a Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Thu, 13 Jun 2024 15:47:27 +0800 Subject: [PATCH 37/37] fix(meta): use per table mce and safe epoch in metadata backup (#17227) --- proto/backup_service.proto | 3 ++ .../rw_catalog/rw_meta_snapshot.rs | 31 +++--------- .../meta_snapshot_builder_v2.rs | 2 +- .../src/backup_restore/restore_impl/v2.rs | 46 ++++++++++------- .../backup/integration_tests/common.sh | 14 ++---- .../integration_tests/test_query_backup.sh | 10 ++-- src/storage/backup/src/lib.rs | 49 ++++++++++++++++++- src/storage/src/hummock/backup_reader.rs | 12 ++++- .../src/hummock/store/hummock_storage.rs | 6 ++- 9 files changed, 113 insertions(+), 60 deletions(-) diff --git a/proto/backup_service.proto b/proto/backup_service.proto index 48fe46ed7eac2..24d410b38f115 100644 --- a/proto/backup_service.proto +++ b/proto/backup_service.proto @@ -2,6 +2,8 @@ syntax = "proto3"; package backup_service; +import "hummock.proto"; + option java_package = "com.risingwave.proto"; option optimize_for = SPEED; @@ -50,6 +52,7 @@ message MetaSnapshotMetadata { optional uint32 format_version = 5; optional string remarks = 6; optional string rw_version = 7; + map state_table_info = 8; } service BackupService { diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs index f31b1f7c67c5c..51df244f9539c 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::{Fields, Timestamp}; -use risingwave_common::util::epoch::Epoch; +use risingwave_common::types::{Fields, JsonbVal}; use risingwave_frontend_macro::system_catalog; +use serde_json::json; use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; @@ -24,30 +24,13 @@ struct RwMetaSnapshot { #[primary_key] meta_snapshot_id: i64, hummock_version_id: i64, - // the smallest epoch this meta snapshot includes - safe_epoch: i64, - // human-readable timestamp of safe_epoch - safe_epoch_ts: Option, - // the largest epoch this meta snapshot includes - max_committed_epoch: i64, - // human-readable timestamp of max_committed_epoch - max_committed_epoch_ts: Option, remarks: Option, + state_table_info: Option, + rw_version: Option, } #[system_catalog(table, "rw_catalog.rw_meta_snapshot")] async fn read_meta_snapshot(reader: &SysCatalogReaderImpl) -> Result> { - let try_get_date_time = |epoch: u64| { - if epoch == 0 { - return None; - } - let time_millis = Epoch::from(epoch).as_unix_millis(); - Timestamp::with_secs_nsecs( - (time_millis / 1000) as i64, - (time_millis % 1000 * 1_000_000) as u32, - ) - .ok() - }; let meta_snapshots = reader .meta_client .list_meta_snapshots() @@ -56,11 +39,9 @@ async fn read_meta_snapshot(reader: &SysCatalogReaderImpl) -> Result BackupError { macro_rules! define_set_metadata { ($( {$name:ident, $mod_path:ident::$mod_name:ident} ),*) => { - pub async fn set_metadata( + async fn set_metadata( metadata: &mut MetadataV2, txn: &sea_orm::DatabaseTransaction, ) -> BackupResult<()> { diff --git a/src/meta/src/backup_restore/restore_impl/v2.rs b/src/meta/src/backup_restore/restore_impl/v2.rs index 13492c56316a2..a431b455063bb 100644 --- a/src/meta/src/backup_restore/restore_impl/v2.rs +++ b/src/meta/src/backup_restore/restore_impl/v2.rs @@ -93,28 +93,40 @@ impl WriterModelV2ToMetaStoreV2 { } } -macro_rules! define_write_model_v2_to_meta_store_v2 { - ($( {$name:ident, $mod_path:ident::$mod_name:ident} ),*) => { - async fn write_model_v2_to_meta_store_v2( - metadata: &risingwave_backup::meta_snapshot_v2::MetadataV2, - db: &sea_orm::DatabaseConnection, - ) -> BackupResult<()> { - $( - insert_models(metadata.$name.clone(), db).await?; - )* - Ok(()) - } - }; -} - -risingwave_backup::for_all_metadata_models_v2!(define_write_model_v2_to_meta_store_v2); - #[async_trait::async_trait] impl Writer for WriterModelV2ToMetaStoreV2 { async fn write(&self, target_snapshot: MetaSnapshot) -> BackupResult<()> { let metadata = target_snapshot.metadata; let db = &self.meta_store.conn; - write_model_v2_to_meta_store_v2(&metadata, db).await?; + insert_models(metadata.seaql_migrations.clone(), db).await?; + insert_models(metadata.clusters.clone(), db).await?; + insert_models(metadata.version_stats.clone(), db).await?; + insert_models(metadata.compaction_configs.clone(), db).await?; + insert_models(metadata.hummock_sequences.clone(), db).await?; + insert_models(metadata.workers.clone(), db).await?; + insert_models(metadata.worker_properties.clone(), db).await?; + insert_models(metadata.users.clone(), db).await?; + insert_models(metadata.user_privileges.clone(), db).await?; + insert_models(metadata.objects.clone(), db).await?; + insert_models(metadata.object_dependencies.clone(), db).await?; + insert_models(metadata.databases.clone(), db).await?; + insert_models(metadata.schemas.clone(), db).await?; + insert_models(metadata.streaming_jobs.clone(), db).await?; + insert_models(metadata.fragments.clone(), db).await?; + insert_models(metadata.actors.clone(), db).await?; + insert_models(metadata.actor_dispatchers.clone(), db).await?; + insert_models(metadata.connections.clone(), db).await?; + insert_models(metadata.sources.clone(), db).await?; + insert_models(metadata.tables.clone(), db).await?; + insert_models(metadata.sinks.clone(), db).await?; + insert_models(metadata.views.clone(), db).await?; + insert_models(metadata.indexes.clone(), db).await?; + insert_models(metadata.functions.clone(), db).await?; + insert_models(metadata.system_parameters.clone(), db).await?; + insert_models(metadata.catalog_versions.clone(), db).await?; + insert_models(metadata.subscriptions.clone(), db).await?; + insert_models(metadata.session_parameters.clone(), db).await?; + insert_models(metadata.secrets.clone(), db).await?; // update_auto_inc must be called last. update_auto_inc(&metadata, db).await?; Ok(()) diff --git a/src/storage/backup/integration_tests/common.sh b/src/storage/backup/integration_tests/common.sh index d37ea0e7b3c6a..68f37e5d67249 100644 --- a/src/storage/backup/integration_tests/common.sh +++ b/src/storage/backup/integration_tests/common.sh @@ -156,15 +156,15 @@ function execute_sql_and_expect() { } function get_max_committed_epoch() { - mce=$(${BACKUP_TEST_RW_ALL_IN_ONE} risectl hummock list-version --verbose 2>&1 | grep max_committed_epoch | sed -n 's/^.*max_committed_epoch: \(.*\),/\1/p') + mce=$(${BACKUP_TEST_RW_ALL_IN_ONE} risectl hummock list-version --verbose 2>&1 | grep committed_epoch | sed -n 's/^.*committed_epoch: \(.*\),/\1/p') # always take the smallest one echo "${mce}"|sort -n |head -n 1 } function get_safe_epoch() { safe_epoch=$(${BACKUP_TEST_RW_ALL_IN_ONE} risectl hummock list-version --verbose 2>&1 | grep safe_epoch | sed -n 's/^.*safe_epoch: \(.*\),/\1/p') - # always take the smallest one - echo "${safe_epoch}"|sort -n |head -n 1 + # always take the largest one + echo "${safe_epoch}"|sort -n -r |head -n 1 } function get_total_sst_count() { @@ -173,17 +173,13 @@ function get_total_sst_count() { } function get_max_committed_epoch_in_backup() { - local id - id=$1 - sed_str="s/.*{\"id\":${id},\"hummock_version_id\":.*,\"ssts\":\[.*\],\"max_committed_epoch\":\([[:digit:]]*\),\"safe_epoch\":.*}.*/\1/p" + sed_str="s/.*\"state_table_info\":{\"[[:digit:]]*\":{\"committedEpoch\":\"\([[:digit:]]*\)\",\"safeEpoch\":\"\([[:digit:]]*\)\"}.*/\1/p" ${BACKUP_TEST_MCLI} -C "${BACKUP_TEST_MCLI_CONFIG}" \ cat "hummock-minio/hummock001/backup/manifest.json" | sed -n "${sed_str}" } function get_safe_epoch_in_backup() { - local id - id=$1 - sed_str="s/.*{\"id\":${id},\"hummock_version_id\":.*,\"ssts\":\[.*\],\"max_committed_epoch\":.*,\"safe_epoch\":\([[:digit:]]*\).*}.*/\1/p" + sed_str="s/.*\"state_table_info\":{\"[[:digit:]]*\":{\"committedEpoch\":\"\([[:digit:]]*\)\",\"safeEpoch\":\"\([[:digit:]]*\)\"}.*/\2/p" ${BACKUP_TEST_MCLI} -C "${BACKUP_TEST_MCLI_CONFIG}" \ cat "hummock-minio/hummock001/backup/manifest.json" | sed -n "${sed_str}" } diff --git a/src/storage/backup/integration_tests/test_query_backup.sh b/src/storage/backup/integration_tests/test_query_backup.sh index b08216267f27d..dbba68c7e8564 100644 --- a/src/storage/backup/integration_tests/test_query_backup.sh +++ b/src/storage/backup/integration_tests/test_query_backup.sh @@ -24,8 +24,8 @@ select * from t1; job_id=$(backup) echo "${job_id}" -backup_mce=$(get_max_committed_epoch_in_backup "${job_id}") -backup_safe_epoch=$(get_safe_epoch_in_backup "${job_id}") +backup_mce=$(get_max_committed_epoch_in_backup) +backup_safe_epoch=$(get_safe_epoch_in_backup) echo "backup MCE: ${backup_mce}" echo "backup safe_epoch: ${backup_safe_epoch}" @@ -55,10 +55,10 @@ do sleep 5 min_pinned_snapshot=$(get_min_pinned_snapshot) done -# safe epoch equals to 0 because no compaction has been done +# safe epoch equals to backup_safe_epoch because no compaction has been done safe_epoch=$(get_safe_epoch) echo "safe epoch after unpin: ${safe_epoch}" -[ "${safe_epoch}" -eq 0 ] +[ "${safe_epoch}" -eq "${backup_safe_epoch}" ] # trigger a compaction to increase safe_epoch manual_compaction -c 3 -l 0 # wait until compaction is done @@ -68,6 +68,7 @@ do sleep 5 done echo "safe epoch after compaction: ${safe_epoch}" +[ "${safe_epoch}" -gt "${backup_safe_epoch}" ] echo "QUERY_EPOCH=safe_epoch. It should fail because it's not covered by any backup" execute_sql_and_expect \ @@ -83,7 +84,6 @@ select * from t1;" \ echo "QUERY_EPOCH=backup_safe_epoch + 1<<16 + 1, it's < safe_epoch but covered by backup" epoch=$((backup_safe_epoch + (1<<16) + 1)) -[ ${epoch} -eq 65537 ] execute_sql_and_expect \ "SET QUERY_EPOCH TO ${epoch}; select * from t1;" \ diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index ed8dccf8d1e49..6b569277c54dd 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -32,14 +32,16 @@ pub mod meta_snapshot_v1; pub mod meta_snapshot_v2; pub mod storage; -use std::collections::HashSet; +use std::collections::{HashMap, HashSet}; use std::hash::Hasher; use itertools::Itertools; +use risingwave_common::catalog::TableId; use risingwave_common::RW_VERSION; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{HummockSstableObjectId, HummockVersionId}; use risingwave_pb::backup_service::{PbMetaSnapshotManifest, PbMetaSnapshotMetadata}; +use risingwave_pb::hummock::PbStateTableInfo; use serde::{Deserialize, Serialize}; use crate::error::{BackupError, BackupResult}; @@ -58,6 +60,8 @@ pub struct MetaSnapshotMetadata { #[serde(default)] pub format_version: u32, pub remarks: Option, + #[serde(with = "table_id_key_map")] + pub state_table_info: HashMap, pub rw_version: Option, } @@ -76,6 +80,7 @@ impl MetaSnapshotMetadata { safe_epoch: v.visible_table_safe_epoch(), format_version, remarks, + state_table_info: v.state_table_info.info().clone(), rw_version: Some(RW_VERSION.to_owned()), } } @@ -115,6 +120,11 @@ impl From<&MetaSnapshotMetadata> for PbMetaSnapshotMetadata { safe_epoch: m.safe_epoch, format_version: Some(m.format_version), remarks: m.remarks.clone(), + state_table_info: m + .state_table_info + .iter() + .map(|(t, i)| (t.table_id, i.clone())) + .collect(), rw_version: m.rw_version.clone(), } } @@ -128,3 +138,40 @@ impl From<&MetaSnapshotManifest> for PbMetaSnapshotManifest { } } } + +mod table_id_key_map { + use std::collections::HashMap; + use std::str::FromStr; + + use risingwave_common::catalog::TableId; + use risingwave_pb::hummock::PbStateTableInfo; + use serde::{Deserialize, Deserializer, Serialize, Serializer}; + + pub fn serialize( + map: &HashMap, + serializer: S, + ) -> Result + where + S: Serializer, + { + let map_as_str: HashMap = + map.iter().map(|(k, v)| (k.to_string(), v)).collect(); + map_as_str.serialize(serializer) + } + + pub fn deserialize<'de, D>( + deserializer: D, + ) -> Result, D::Error> + where + D: Deserializer<'de>, + { + let map_as_str: HashMap = HashMap::deserialize(deserializer)?; + map_as_str + .into_iter() + .map(|(k, v)| { + let key = u32::from_str(&k).map_err(serde::de::Error::custom)?; + Ok((TableId::new(key), v)) + }) + .collect() + } +} diff --git a/src/storage/src/hummock/backup_reader.rs b/src/storage/src/hummock/backup_reader.rs index da1707db3ade1..3fa0748b21737 100644 --- a/src/storage/src/hummock/backup_reader.rs +++ b/src/storage/src/hummock/backup_reader.rs @@ -25,6 +25,7 @@ use risingwave_backup::error::BackupError; use risingwave_backup::meta_snapshot::{MetaSnapshot, Metadata}; use risingwave_backup::storage::{MetaSnapshotStorage, ObjectStoreMetaSnapshotStorage}; use risingwave_backup::{meta_snapshot_v1, meta_snapshot_v2, MetaSnapshotId}; +use risingwave_common::catalog::TableId; use risingwave_common::config::ObjectStoreConfig; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; @@ -182,6 +183,7 @@ impl BackupReader { /// Otherwise, reading the version may encounter object store error, due to SST absence. pub async fn try_get_hummock_version( self: &BackupReaderRef, + table_id: TableId, epoch: u64, ) -> StorageResult> { // Use the same store throughout the call. @@ -192,7 +194,15 @@ impl BackupReader { .manifest() .snapshot_metadata .iter() - .find(|v| epoch >= v.safe_epoch && epoch <= v.max_committed_epoch) + .find(|v| { + if v.state_table_info.is_empty() { + return epoch >= v.safe_epoch && epoch <= v.max_committed_epoch; + } + if let Some(m) = v.state_table_info.get(&table_id) { + return epoch >= m.safe_epoch && epoch <= m.committed_epoch; + } + false + }) .cloned() else { return Ok(None); diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 2d89fdd401fa2..7f9f956b62ddf 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -311,7 +311,11 @@ impl HummockStorage { table_id: TableId, key_range: TableKeyRange, ) -> StorageResult<(TableKeyRange, ReadVersionTuple)> { - match self.backup_reader.try_get_hummock_version(epoch).await { + match self + .backup_reader + .try_get_hummock_version(table_id, epoch) + .await + { Ok(Some(backup_version)) => { validate_safe_epoch(backup_version.version(), table_id, epoch)?;