From bc43c393446ef402a5ca102e9d2107a54465912c Mon Sep 17 00:00:00 2001 From: Rafael Acevedo Date: Mon, 25 Sep 2023 06:17:47 -0300 Subject: [PATCH 01/17] feat(sink): support compression codec in kafka sink (#12439) --- .github/workflows/hakari_fix.yml | 3 ++- Cargo.lock | 2 ++ src/connector/Cargo.toml | 2 ++ src/connector/src/sink/kafka.rs | 35 ++++++++++++++++++++++++++++++++ 4 files changed, 41 insertions(+), 1 deletion(-) diff --git a/.github/workflows/hakari_fix.yml b/.github/workflows/hakari_fix.yml index 670ca38cccc27..b8ded582c36e9 100644 --- a/.github/workflows/hakari_fix.yml +++ b/.github/workflows/hakari_fix.yml @@ -15,7 +15,8 @@ jobs: steps: - uses: actions/checkout@v3 with: - ref: ${{ github.head_ref }} + ref: ${{ github.event.pull_request.head.ref }} + repository: ${{ github.event.pull_request.head.repo.full_name }} - name: Install cargo-hakari uses: taiki-e/install-action@v2 diff --git a/Cargo.lock b/Cargo.lock index a785a3728e873..a7caa3148b99c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6934,6 +6934,8 @@ dependencies = [ "serde_json", "serde_with 3.3.0", "simd-json", + "strum 0.25.0", + "strum_macros 0.25.2", "tempfile", "thiserror", "time", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index bca5acedbba40..8ae92464caaaa 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -102,6 +102,8 @@ serde_derive = "1" serde_json = "1" serde_with = { version = "3", features = ["json"] } simd-json = "0.10.6" +strum = "0.25" +strum_macros = "0.25" tempfile = "3" thiserror = "1" time = "0.3.28" diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index 46bed8fb0be1a..4ae886d15ce0c 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -29,6 +29,7 @@ use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; use serde_derive::{Deserialize, Serialize}; use serde_with::{serde_as, DisplayFromStr}; +use strum_macros::{Display, EnumString}; use super::encoder::{JsonEncoder, TimestampHandlingMode}; use super::formatter::{ @@ -70,6 +71,16 @@ const fn _default_force_append_only() -> bool { false } +#[derive(Debug, Clone, PartialEq, Display, Serialize, Deserialize, EnumString)] +#[strum(serialize_all = "snake_case")] +enum CompressionCodec { + None, + Gzip, + Snappy, + Lz4, + Zstd, +} + #[serde_as] #[derive(Debug, Clone, Serialize, Deserialize)] pub struct RdKafkaPropertiesProducer { @@ -125,6 +136,11 @@ pub struct RdKafkaPropertiesProducer { #[serde(rename = "properties.batch.size")] #[serde_as(as = "Option")] batch_size: Option, + + /// Compression codec to use for compressing message sets. + #[serde(rename = "properties.compression.codec")] + #[serde_as(as = "Option")] + compression_codec: Option, } impl RdKafkaPropertiesProducer { @@ -153,6 +169,9 @@ impl RdKafkaPropertiesProducer { if let Some(v) = self.batch_size { c.set("batch.size", v.to_string()); } + if let Some(v) = &self.compression_codec { + c.set("compression.codec", v.to_string()); + } } } @@ -650,12 +669,17 @@ mod test { "properties.retry.backoff.ms".to_string() => "114514".to_string(), "properties.batch.num.messages".to_string() => "114514".to_string(), "properties.batch.size".to_string() => "114514".to_string(), + "properties.compression.codec".to_string() => "zstd".to_string(), }; let c = KafkaConfig::from_hashmap(props).unwrap(); assert_eq!( c.rdkafka_properties.queue_buffering_max_ms, Some(114.514f64) ); + assert_eq!( + c.rdkafka_properties.compression_codec, + Some(CompressionCodec::Zstd) + ); let props: HashMap = hashmap! { // basic @@ -677,6 +701,16 @@ mod test { "properties.queue.buffering.max.kbytes".to_string() => "-114514".to_string(), // usize cannot be negative }; assert!(KafkaConfig::from_hashmap(props).is_err()); + + let props: HashMap = hashmap! { + // basic + "connector".to_string() => "kafka".to_string(), + "properties.bootstrap.server".to_string() => "localhost:9092".to_string(), + "topic".to_string() => "test".to_string(), + "type".to_string() => "append-only".to_string(), + "properties.compression.codec".to_string() => "notvalid".to_string(), // has to be a valid CompressionCodec + }; + assert!(KafkaConfig::from_hashmap(props).is_err()); } #[test] @@ -762,6 +796,7 @@ mod test { "properties.bootstrap.server".to_string() => "localhost:29092".to_string(), "type".to_string() => "append-only".to_string(), "topic".to_string() => "test_topic".to_string(), + "properties.compression.codec".to_string() => "zstd".to_string(), }; // Create a table with two columns (| id : INT32 | v2 : VARCHAR |) here From 79b5c4ccdebf74df9d0e6e0b66943bcedd06ca94 Mon Sep 17 00:00:00 2001 From: Li0k Date: Mon, 25 Sep 2023 17:32:53 +0800 Subject: [PATCH 02/17] fix(storage): fix condition of compaction task validator (#12473) --- .../picker/base_level_compaction_picker.rs | 3 +- .../picker/compaction_task_validator.rs | 40 +++++++++++-------- 2 files changed, 26 insertions(+), 17 deletions(-) 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 d9bd8d6020a23..684a6a3125bb8 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 @@ -137,7 +137,7 @@ impl LevelCompactionPicker { min_compaction_bytes, // divide by 2 because we need to select files of base level and it need use the other // half quota. - std::cmp::min( + std::cmp::max( self.config.max_bytes_for_level_base, self.config.max_compaction_bytes / 2, ), @@ -573,6 +573,7 @@ pub mod tests { let config = Arc::new( CompactionConfigBuilder::new() .max_compaction_bytes(100010) + .max_bytes_for_level_base(512) .level0_sub_level_compact_level_count(1) .build(), ); 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 eafe074a88288..7452f65d6503a 100644 --- a/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs +++ b/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs @@ -89,32 +89,32 @@ struct TierCompactionTaskValidationRule { impl CompactionTaskValidationRule for TierCompactionTaskValidationRule { fn validate(&self, input: &CompactionInput, stats: &mut LocalPickerStatistic) -> bool { - // so the design here wants to merge multiple overlapping-levels in one compaction - let max_compaction_bytes = std::cmp::min( - self.config.max_compaction_bytes, - self.config.sub_level_max_compaction_bytes - * self.config.level0_overlapping_sub_level_compact_level_count 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 waiting_enough_files = { - if input.select_input_size > max_compaction_bytes { - false - } else { - input.total_file_count <= overlapping_max_compact_file_numer - } - }; + if input.total_file_count >= overlapping_max_compact_file_numer + || input.input_levels.len() >= MAX_COMPACT_LEVEL_COUNT + { + return true; + } + + // so the design here wants to merge multiple overlapping-levels in one compaction + let max_compaction_bytes = std::cmp::min( + self.config.max_compaction_bytes, + self.config.sub_level_max_compaction_bytes + * self.config.level0_overlapping_sub_level_compact_level_count as u64, + ); // If waiting_enough_files is not satisfied, we will raise the priority of the number of // levels to ensure that we can merge as many sub_levels as possible let tier_sub_level_compact_level_count = self.config.level0_overlapping_sub_level_compact_level_count as usize; - if input.input_levels.len() < tier_sub_level_compact_level_count && waiting_enough_files { + if input.input_levels.len() < tier_sub_level_compact_level_count + && input.select_input_size < max_compaction_bytes + { stats.skip_by_count_limit += 1; return false; } @@ -129,7 +129,9 @@ struct IntraCompactionTaskValidationRule { 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 { + if input.total_file_count >= self.config.level0_max_compact_file_number + || input.input_levels.len() >= MAX_COMPACT_LEVEL_COUNT + { return true; } @@ -175,6 +177,12 @@ 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 + { + return true; + } + // The size of target level may be too large, we shall skip this compact task and wait // the data in base level compact to lower level. if input.target_input_size > self.config.max_compaction_bytes { From a0f0b6a6ee60c7664f2a2a081fe7a7585ec6d2c7 Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 25 Sep 2023 20:30:36 +0800 Subject: [PATCH 03/17] ci: increase timeout for unit test/buid-other (#12498) --- ci/workflows/pull-request.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 1727528c5946f..d289dd130b8ae 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -40,7 +40,7 @@ steps: mount-buildkite-agent: true environment: - GITHUB_TOKEN - timeout_in_minutes: 12 + timeout_in_minutes: 14 retry: *auto-retry - label: "build (deterministic simulation)" @@ -273,7 +273,7 @@ steps: config: ci/docker-compose.yml environment: - CODECOV_TOKEN - timeout_in_minutes: 18 + timeout_in_minutes: 20 retry: *auto-retry - label: "check" From 9599be0ac5fa2c7ed57e877d6f30de7ba4fa3d12 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Mon, 25 Sep 2023 21:08:12 +0800 Subject: [PATCH 04/17] refactor(optimizer): rename logical pk to stream key (#12516) --- .../src/optimizer/plan_node/derive.rs | 2 +- .../src/optimizer/plan_node/generic/agg.rs | 4 +- .../src/optimizer/plan_node/generic/dedup.rs | 2 +- .../src/optimizer/plan_node/generic/delete.rs | 4 +- .../plan_node/generic/dynamic_filter.rs | 6 +- .../src/optimizer/plan_node/generic/except.rs | 4 +- .../src/optimizer/plan_node/generic/expand.rs | 4 +- .../src/optimizer/plan_node/generic/filter.rs | 4 +- .../optimizer/plan_node/generic/hop_window.rs | 4 +- .../src/optimizer/plan_node/generic/insert.rs | 2 +- .../optimizer/plan_node/generic/intersect.rs | 4 +- .../src/optimizer/plan_node/generic/join.rs | 6 +- .../src/optimizer/plan_node/generic/limit.rs | 4 +- .../src/optimizer/plan_node/generic/mod.rs | 8 +-- .../plan_node/generic/over_window.rs | 4 +- .../optimizer/plan_node/generic/project.rs | 4 +- .../plan_node/generic/project_set.rs | 4 +- .../src/optimizer/plan_node/generic/scan.rs | 4 +- .../src/optimizer/plan_node/generic/share.rs | 4 +- .../src/optimizer/plan_node/generic/source.rs | 4 +- .../src/optimizer/plan_node/generic/top_n.rs | 4 +- .../src/optimizer/plan_node/generic/union.rs | 4 +- .../src/optimizer/plan_node/generic/update.rs | 4 +- .../src/optimizer/plan_node/logical_agg.rs | 2 +- .../src/optimizer/plan_node/logical_apply.rs | 2 +- .../optimizer/plan_node/logical_hop_window.rs | 6 +- .../src/optimizer/plan_node/logical_join.rs | 8 +-- .../optimizer/plan_node/logical_multi_join.rs | 2 +- .../optimizer/plan_node/logical_project.rs | 2 +- .../plan_node/logical_project_set.rs | 2 +- .../src/optimizer/plan_node/logical_scan.rs | 2 +- .../src/optimizer/plan_node/logical_topn.rs | 2 +- .../src/optimizer/plan_node/logical_union.rs | 10 +-- src/frontend/src/optimizer/plan_node/mod.rs | 10 +-- .../src/optimizer/plan_node/plan_base.rs | 28 ++++---- .../src/optimizer/plan_node/stream.rs | 18 ++--- .../src/optimizer/plan_node/stream_derive.rs | 68 +++++++++---------- .../src/optimizer/plan_node/stream_dml.rs | 2 +- .../plan_node/stream_eowc_over_window.rs | 2 +- .../optimizer/plan_node/stream_exchange.rs | 4 +- .../optimizer/plan_node/stream_group_topn.rs | 2 +- .../optimizer/plan_node/stream_materialize.rs | 4 +- .../optimizer/plan_node/stream_over_window.rs | 2 +- .../optimizer/plan_node/stream_row_id_gen.rs | 2 +- .../src/optimizer/plan_node/stream_share.rs | 2 +- .../src/optimizer/plan_node/stream_sink.rs | 2 +- .../src/optimizer/plan_node/stream_sort.rs | 6 +- .../optimizer/plan_node/stream_table_scan.rs | 4 +- .../src/optimizer/plan_node/stream_topn.rs | 6 +- .../src/optimizer/plan_node/stream_values.rs | 2 +- .../plan_node/stream_watermark_filter.rs | 2 +- .../plan_visitor/cardinality_visitor.rs | 2 +- 52 files changed, 150 insertions(+), 150 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/derive.rs b/src/frontend/src/optimizer/plan_node/derive.rs index fe2bd3e2da325..7153f8e6a8943 100644 --- a/src/frontend/src/optimizer/plan_node/derive.rs +++ b/src/frontend/src/optimizer/plan_node/derive.rs @@ -82,7 +82,7 @@ pub(crate) fn derive_pk( columns: &[ColumnCatalog], ) -> (Vec, Vec) { // Note(congyi): avoid pk duplication - let stream_key = input.logical_pk().iter().copied().unique().collect_vec(); + let stream_key = input.stream_key().iter().copied().unique().collect_vec(); let schema = input.schema(); // Assert the uniqueness of column names and IDs, including hidden columns. diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 86aee0c01a7c1..b345c0b5f0206 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -195,7 +195,7 @@ impl GenericPlanNode for Agg { Schema { fields } } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { Some((0..self.group_key.len()).collect()) } @@ -344,7 +344,7 @@ impl Agg { window_col_idx: Option, ) -> Vec { let in_fields = self.input.schema().fields().to_vec(); - let in_pks = self.input.logical_pk().to_vec(); + let in_pks = self.input.stream_key().to_vec(); let in_append_only = self.input.append_only(); let in_dist_key = self.input.distribution().dist_column_indices().to_vec(); diff --git a/src/frontend/src/optimizer/plan_node/generic/dedup.rs b/src/frontend/src/optimizer/plan_node/generic/dedup.rs index 89bfe14b2e9d3..bcec1b41ff7da 100644 --- a/src/frontend/src/optimizer/plan_node/generic/dedup.rs +++ b/src/frontend/src/optimizer/plan_node/generic/dedup.rs @@ -50,7 +50,7 @@ impl GenericPlanNode for Dedup { self.input.schema().clone() } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { Some(self.dedup_cols.clone()) } diff --git a/src/frontend/src/optimizer/plan_node/generic/delete.rs b/src/frontend/src/optimizer/plan_node/generic/delete.rs index 5d178b654acec..23ab1caf6f442 100644 --- a/src/frontend/src/optimizer/plan_node/generic/delete.rs +++ b/src/frontend/src/optimizer/plan_node/generic/delete.rs @@ -60,9 +60,9 @@ impl GenericPlanNode for Delete { } } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { if self.returning { - Some(self.input.logical_pk().to_vec()) + Some(self.input.stream_key().to_vec()) } else { Some(vec![]) } diff --git a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs index aa8afc1779be8..1e45d8f710af2 100644 --- a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs @@ -58,8 +58,8 @@ impl GenericPlanNode for DynamicFilter { self.left.schema().clone() } - fn logical_pk(&self) -> Option> { - Some(self.left.logical_pk().to_vec()) + fn stream_key(&self) -> Option> { + Some(self.left.stream_key().to_vec()) } fn ctx(&self) -> OptimizerContextRef { @@ -151,7 +151,7 @@ pub fn infer_left_internal_table_catalog( let mut pk_indices = vec![left_key_index]; let read_prefix_len_hint = pk_indices.len(); - for i in me.logical_pk() { + for i in me.stream_key() { if *i != left_key_index { pk_indices.push(*i); } diff --git a/src/frontend/src/optimizer/plan_node/generic/except.rs b/src/frontend/src/optimizer/plan_node/generic/except.rs index 3721db69eefb8..7dc99c8290210 100644 --- a/src/frontend/src/optimizer/plan_node/generic/except.rs +++ b/src/frontend/src/optimizer/plan_node/generic/except.rs @@ -33,8 +33,8 @@ impl GenericPlanNode for Except { self.inputs[0].schema().clone() } - fn logical_pk(&self) -> Option> { - Some(self.inputs[0].logical_pk().to_vec()) + fn stream_key(&self) -> Option> { + Some(self.inputs[0].stream_key().to_vec()) } fn ctx(&self) -> OptimizerContextRef { diff --git a/src/frontend/src/optimizer/plan_node/generic/expand.rs b/src/frontend/src/optimizer/plan_node/generic/expand.rs index d78bd4c112028..55624b04baa99 100644 --- a/src/frontend/src/optimizer/plan_node/generic/expand.rs +++ b/src/frontend/src/optimizer/plan_node/generic/expand.rs @@ -57,11 +57,11 @@ impl GenericPlanNode for Expand { Schema::new(fields) } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { let input_schema_len = self.input.schema().len(); let mut pk_indices = self .input - .logical_pk() + .stream_key() .iter() .map(|&pk| pk + input_schema_len) .collect_vec(); diff --git a/src/frontend/src/optimizer/plan_node/generic/filter.rs b/src/frontend/src/optimizer/plan_node/generic/filter.rs index 2f6542cb19a9a..bac03135c4214 100644 --- a/src/frontend/src/optimizer/plan_node/generic/filter.rs +++ b/src/frontend/src/optimizer/plan_node/generic/filter.rs @@ -53,8 +53,8 @@ impl GenericPlanNode for Filter { self.input.schema().clone() } - fn logical_pk(&self) -> Option> { - Some(self.input.logical_pk().to_vec()) + fn stream_key(&self) -> Option> { + Some(self.input.stream_key().to_vec()) } fn ctx(&self) -> OptimizerContextRef { diff --git a/src/frontend/src/optimizer/plan_node/generic/hop_window.rs b/src/frontend/src/optimizer/plan_node/generic/hop_window.rs index 131e7c6f1455e..37d9401ee194d 100644 --- a/src/frontend/src/optimizer/plan_node/generic/hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/generic/hop_window.rs @@ -63,7 +63,7 @@ impl GenericPlanNode for HopWindow { .collect() } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { let window_start_index = self .output_indices .iter() @@ -77,7 +77,7 @@ impl GenericPlanNode for HopWindow { } else { let mut pk = self .input - .logical_pk() + .stream_key() .iter() .filter_map(|&pk_idx| self.output_indices.iter().position(|&idx| idx == pk_idx)) .collect_vec(); diff --git a/src/frontend/src/optimizer/plan_node/generic/insert.rs b/src/frontend/src/optimizer/plan_node/generic/insert.rs index c5bfeb725ff83..727e0296e0c83 100644 --- a/src/frontend/src/optimizer/plan_node/generic/insert.rs +++ b/src/frontend/src/optimizer/plan_node/generic/insert.rs @@ -61,7 +61,7 @@ impl GenericPlanNode for Insert { } } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { None } diff --git a/src/frontend/src/optimizer/plan_node/generic/intersect.rs b/src/frontend/src/optimizer/plan_node/generic/intersect.rs index 3b781eeb37fbd..84c34e0b8d507 100644 --- a/src/frontend/src/optimizer/plan_node/generic/intersect.rs +++ b/src/frontend/src/optimizer/plan_node/generic/intersect.rs @@ -32,8 +32,8 @@ impl GenericPlanNode for Intersect { self.inputs[0].schema().clone() } - fn logical_pk(&self) -> Option> { - Some(self.inputs[0].logical_pk().to_vec()) + fn stream_key(&self) -> Option> { + Some(self.inputs[0].stream_key().to_vec()) } fn ctx(&self) -> OptimizerContextRef { diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index 47c6b66286d98..d4cc84376599f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -90,11 +90,11 @@ impl GenericPlanNode for Join { Schema { fields } } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { let _left_len = self.left.schema().len(); let _right_len = self.right.schema().len(); - let left_pk = self.left.logical_pk(); - let right_pk = self.right.logical_pk(); + let left_pk = self.left.stream_key(); + let right_pk = self.right.stream_key(); let l2i = self.l2i_col_mapping(); let r2i = self.r2i_col_mapping(); let full_out_col_num = self.internal_column_num(); diff --git a/src/frontend/src/optimizer/plan_node/generic/limit.rs b/src/frontend/src/optimizer/plan_node/generic/limit.rs index 060fdf47cbda4..2773ea325285e 100644 --- a/src/frontend/src/optimizer/plan_node/generic/limit.rs +++ b/src/frontend/src/optimizer/plan_node/generic/limit.rs @@ -42,8 +42,8 @@ impl GenericPlanNode for Limit { self.input.functional_dependency().clone() } - fn logical_pk(&self) -> Option> { - Some(self.input.logical_pk().to_vec()) + fn stream_key(&self) -> Option> { + Some(self.input.stream_key().to_vec()) } } impl Limit { diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index 032eaa40fcda2..cc02763f26f98 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -86,22 +86,22 @@ pub(super) use impl_distill_unit_from_fields; pub trait GenericPlanRef: Eq + Hash { fn schema(&self) -> &Schema; - fn logical_pk(&self) -> &[usize]; + fn stream_key(&self) -> &[usize]; fn functional_dependency(&self) -> &FunctionalDependencySet; fn ctx(&self) -> OptimizerContextRef; } pub trait GenericPlanNode { - /// return (schema, `logical_pk`, fds) + /// return (schema, `stream_key`, fds) fn logical_properties(&self) -> (Schema, Option>, FunctionalDependencySet) { ( self.schema(), - self.logical_pk(), + self.stream_key(), self.functional_dependency(), ) } fn functional_dependency(&self) -> FunctionalDependencySet; fn schema(&self) -> Schema; - fn logical_pk(&self) -> Option>; + fn stream_key(&self) -> Option>; fn ctx(&self) -> OptimizerContextRef; } diff --git a/src/frontend/src/optimizer/plan_node/generic/over_window.rs b/src/frontend/src/optimizer/plan_node/generic/over_window.rs index c148711698a24..cf7bc028ff82f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/over_window.rs +++ b/src/frontend/src/optimizer/plan_node/generic/over_window.rs @@ -218,8 +218,8 @@ impl GenericPlanNode for OverWindow { schema } - fn logical_pk(&self) -> Option> { - let mut output_pk = self.input.logical_pk().to_vec(); + fn stream_key(&self) -> Option> { + let mut output_pk = self.input.stream_key().to_vec(); for part_key_idx in self .window_functions .iter() diff --git a/src/frontend/src/optimizer/plan_node/generic/project.rs b/src/frontend/src/optimizer/plan_node/generic/project.rs index 1d35332f0e709..ca1dc7ae4468f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project.rs @@ -97,10 +97,10 @@ impl GenericPlanNode for Project { Schema { fields } } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { let i2o = self.i2o_col_mapping(); self.input - .logical_pk() + .stream_key() .iter() .map(|pk_col| i2o.try_map(*pk_col)) .collect::>>() diff --git a/src/frontend/src/optimizer/plan_node/generic/project_set.rs b/src/frontend/src/optimizer/plan_node/generic/project_set.rs index e159927f7bfda..3e5d5585f782c 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project_set.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project_set.rs @@ -88,11 +88,11 @@ impl GenericPlanNode for ProjectSet { Schema { fields } } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { let i2o = self.i2o_col_mapping(); let mut pk = self .input - .logical_pk() + .stream_key() .iter() .map(|pk_col| i2o.try_map(*pk_col)) .collect::>>() diff --git a/src/frontend/src/optimizer/plan_node/generic/scan.rs b/src/frontend/src/optimizer/plan_node/generic/scan.rs index 526a677e279b5..b7de99d11096b 100644 --- a/src/frontend/src/optimizer/plan_node/generic/scan.rs +++ b/src/frontend/src/optimizer/plan_node/generic/scan.rs @@ -307,7 +307,7 @@ impl GenericPlanNode for Scan { Schema { fields } } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { let id_to_op_idx = Self::get_id_to_op_idx_mapping(&self.output_col_idx, &self.table_desc); self.table_desc .stream_key @@ -325,7 +325,7 @@ impl GenericPlanNode for Scan { } fn functional_dependency(&self) -> FunctionalDependencySet { - let pk_indices = self.logical_pk(); + let pk_indices = self.stream_key(); let col_num = self.output_col_idx.len(); match &pk_indices { Some(pk_indices) => FunctionalDependencySet::with_key(col_num, pk_indices), diff --git a/src/frontend/src/optimizer/plan_node/generic/share.rs b/src/frontend/src/optimizer/plan_node/generic/share.rs index c22a46357fa66..d9c32b6a28f6f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/share.rs +++ b/src/frontend/src/optimizer/plan_node/generic/share.rs @@ -43,8 +43,8 @@ impl GenericPlanNode for Share { self.input.borrow().schema().clone() } - fn logical_pk(&self) -> Option> { - Some(self.input.borrow().logical_pk().to_vec()) + fn stream_key(&self) -> Option> { + Some(self.input.borrow().stream_key().to_vec()) } fn ctx(&self) -> OptimizerContextRef { diff --git a/src/frontend/src/optimizer/plan_node/generic/source.rs b/src/frontend/src/optimizer/plan_node/generic/source.rs index 4a4f092110dee..4d508cc37894e 100644 --- a/src/frontend/src/optimizer/plan_node/generic/source.rs +++ b/src/frontend/src/optimizer/plan_node/generic/source.rs @@ -60,7 +60,7 @@ impl GenericPlanNode for Source { Schema { fields } } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { self.row_id_index.map(|idx| vec![idx]) } @@ -69,7 +69,7 @@ impl GenericPlanNode for Source { } fn functional_dependency(&self) -> FunctionalDependencySet { - let pk_indices = self.logical_pk(); + let pk_indices = self.stream_key(); match pk_indices { Some(pk_indices) => { FunctionalDependencySet::with_key(self.column_catalog.len(), &pk_indices) diff --git a/src/frontend/src/optimizer/plan_node/generic/top_n.rs b/src/frontend/src/optimizer/plan_node/generic/top_n.rs index 990e37219d19b..dcca35f5a5d9f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/top_n.rs +++ b/src/frontend/src/optimizer/plan_node/generic/top_n.rs @@ -170,13 +170,13 @@ impl GenericPlanNode for TopN { self.input.schema().clone() } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { // We can use the group key as the stream key when there is at most one record for each // value of the group key. if self.limit_attr.max_one_row() { Some(self.group_key.clone()) } else { - let mut pk = self.input.logical_pk().to_vec(); + let mut pk = self.input.stream_key().to_vec(); for i in &self.group_key { if !pk.contains(i) { pk.push(*i); diff --git a/src/frontend/src/optimizer/plan_node/generic/union.rs b/src/frontend/src/optimizer/plan_node/generic/union.rs index 91f10eac749f0..b8a1d9e2ec5e3 100644 --- a/src/frontend/src/optimizer/plan_node/generic/union.rs +++ b/src/frontend/src/optimizer/plan_node/generic/union.rs @@ -36,11 +36,11 @@ impl GenericPlanNode for Union { self.inputs[0].schema().clone() } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { // Union all its inputs pks + source_col if exists let mut pk_indices = vec![]; for input in &self.inputs { - for pk in input.logical_pk() { + for pk in input.stream_key() { if !pk_indices.contains(pk) { pk_indices.push(*pk); } diff --git a/src/frontend/src/optimizer/plan_node/generic/update.rs b/src/frontend/src/optimizer/plan_node/generic/update.rs index 0943d770daaef..f832806e7cf9a 100644 --- a/src/frontend/src/optimizer/plan_node/generic/update.rs +++ b/src/frontend/src/optimizer/plan_node/generic/update.rs @@ -62,9 +62,9 @@ impl GenericPlanNode for Update { } } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { if self.returning { - Some(self.input.logical_pk().to_vec()) + Some(self.input.stream_key().to_vec()) } else { Some(vec![]) } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index a2099b7d33f81..e24f060fd9e8a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -222,7 +222,7 @@ impl LogicalAgg { // so it obeys consistent hash strategy via [`Distribution::HashShard`]. let stream_input = if *input_dist == Distribution::SomeShard && self.core.must_try_two_phase_agg() { - RequiredDist::shard_by_key(stream_input.schema().len(), stream_input.logical_pk()) + RequiredDist::shard_by_key(stream_input.schema().len(), stream_input.stream_key()) .enforce_if_not_satisfies(stream_input, &Order::any())? } else { stream_input diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index 0ea21532458fe..3ab29bf4ff4d9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -86,7 +86,7 @@ impl LogicalApply { let ctx = left.ctx(); let join_core = generic::Join::with_full_output(left, right, join_type, on); let schema = join_core.schema(); - let pk_indices = join_core.logical_pk(); + let pk_indices = join_core.stream_key(); let (functional_dependency, pk_indices) = match pk_indices { Some(pk_indices) => ( FunctionalDependencySet::with_key(schema.len(), &pk_indices), diff --git a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs index 3905daaaf9f85..73fd3730b0c4c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -64,7 +64,7 @@ impl LogicalHopWindow { }; let _schema = core.schema(); - let _pk_indices = core.logical_pk(); + let _pk_indices = core.stream_key(); let ctx = core.ctx(); // NOTE(st1page): add join keys in the pk_indices a work around before we really have stream @@ -79,7 +79,7 @@ impl LogicalHopWindow { let base = PlanBase::new_logical( ctx, core.schema(), - core.logical_pk().unwrap_or_default(), + core.stream_key().unwrap_or_default(), core.functional_dependency(), ); @@ -344,7 +344,7 @@ impl ToStream for LogicalHopWindow { let i2o = self.core.i2o_col_mapping(); output_indices.extend( input - .logical_pk() + .stream_key() .iter() .cloned() .filter(|i| i2o.try_map(*i).is_none()), diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 640b31170c546..a9e398350f8d4 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -1395,14 +1395,14 @@ impl ToStream for LogicalJoin { // Add missing pk indices to the logical join let mut left_to_add = left - .logical_pk() + .stream_key() .iter() .cloned() .filter(|i| l2o.try_map(*i).is_none()) .collect_vec(); let mut right_to_add = right - .logical_pk() + .stream_key() .iter() .filter(|&&i| r2o.try_map(i).is_none()) .map(|&i| i + left_len) @@ -1464,13 +1464,13 @@ impl ToStream for LogicalJoin { .composite(&join_with_pk.core.i2o_col_mapping()); let left_right_stream_keys = join_with_pk .left() - .logical_pk() + .stream_key() .iter() .map(|i| l2o.map(*i)) .chain( join_with_pk .right() - .logical_pk() + .stream_key() .iter() .map(|i| r2o.map(*i)), ) diff --git a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs index b3d61cd495fb9..2a74d227dc86c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -246,7 +246,7 @@ impl LogicalMultiJoin { let pk_indices = { let mut pk_indices = vec![]; - for (i, input_pk) in inputs.iter().map(|input| input.logical_pk()).enumerate() { + for (i, input_pk) in inputs.iter().map(|input| input.stream_key()).enumerate() { for input_pk_idx in input_pk { pk_indices.push(inner_i2o_mappings[i].map(*input_pk_idx)); } diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index 6e1f097e3c7d2..0eea08b6ac3ad 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -264,7 +264,7 @@ impl ToStream for LogicalProject { let (proj, out_col_change) = self.rewrite_with_input(input.clone(), input_col_change); // Add missing columns of input_pk into the select list. - let input_pk = input.logical_pk(); + let input_pk = input.stream_key(); let i2o = proj.i2o_col_mapping(); let col_need_to_add = input_pk .iter() diff --git a/src/frontend/src/optimizer/plan_node/logical_project_set.rs b/src/frontend/src/optimizer/plan_node/logical_project_set.rs index 12100631ca666..50ad246b5bb09 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -363,7 +363,7 @@ impl ToStream for LogicalProjectSet { self.rewrite_with_input(input.clone(), input_col_change); // Add missing columns of input_pk into the select list. - let input_pk = input.logical_pk(); + let input_pk = input.stream_key(); let i2o = self.core.i2o_col_mapping(); let col_need_to_add = input_pk .iter() diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index e671f7412c661..8ee3e6504f185 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -549,7 +549,7 @@ impl ToStream for LogicalScan { None.into(), ))); } - match self.base.logical_pk.is_empty() { + match self.base.stream_key.is_empty() { true => { let mut col_ids = HashSet::new(); diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index 8d9d446c0900f..1e4c6113f2ee7 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -177,7 +177,7 @@ impl LogicalTopN { ); // TODO(st1page): solve it - let global_top_n = StreamTopN::with_stream_key(global_top_n, self.logical_pk().to_vec()); + let global_top_n = StreamTopN::with_stream_key(global_top_n, self.stream_key().to_vec()); // use another projection to remove the column we added before. exprs.pop(); diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index 38ef55405693b..2538450606772 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -139,7 +139,7 @@ impl ToBatch for LogicalUnion { impl ToStream for LogicalUnion { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { // TODO: use round robin distribution instead of using hash distribution of all inputs. - let dist = RequiredDist::hash_shard(self.base.logical_pk()); + let dist = RequiredDist::hash_shard(self.base.stream_key()); let new_inputs: Result> = self .inputs() .iter() @@ -174,7 +174,7 @@ impl ToStream for LogicalUnion { .map(|x| col_index_mapping.map(x)) .collect_vec(); new_input - .logical_pk() + .stream_key() .iter() .all(|x| original_schema_new_pos.contains(x)) }); @@ -223,7 +223,7 @@ impl ToStream for LogicalUnion { .iter() .flat_map(|(new_input, _)| { new_input - .logical_pk() + .stream_key() .iter() .map(|x| new_input.schema().fields[*x].data_type()) }) @@ -234,7 +234,7 @@ impl ToStream for LogicalUnion { .collect_vec(); let input_pk_lens = rewrites .iter() - .map(|(new_input, _)| new_input.logical_pk().len()) + .map(|(new_input, _)| new_input.stream_key().len()) .collect_vec(); let mut input_pk_offsets = vec![0]; for (i, len) in input_pk_lens.into_iter().enumerate() { @@ -258,7 +258,7 @@ impl ToStream for LogicalUnion { .collect_vec(); // input1_pk + input2_pk + ... let mut input_pks = input_pk_nulls.clone(); - for (j, pk_idx) in new_input.logical_pk().iter().enumerate() { + for (j, pk_idx) in new_input.stream_key().iter().enumerate() { input_pks[input_pk_offsets[i] + j] = ExprImpl::InputRef( InputRef::new( *pk_idx, diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 926cf85048f3e..189ab9a0f1a6d 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -435,8 +435,8 @@ impl GenericPlanRef for PlanRef { &self.plan_base().schema } - fn logical_pk(&self) -> &[usize] { - &self.plan_base().logical_pk + fn stream_key(&self) -> &[usize] { + &self.plan_base().stream_key } fn ctx(&self) -> OptimizerContextRef { @@ -514,8 +514,8 @@ impl dyn PlanNode { &self.plan_base().schema } - pub fn logical_pk(&self) -> &[usize] { - &self.plan_base().logical_pk + pub fn stream_key(&self) -> &[usize] { + &self.plan_base().stream_key } pub fn order(&self) -> &Order { @@ -566,7 +566,7 @@ impl dyn PlanNode { identity: self.explain_myself_to_string(), node_body: node, operator_id: self.id().0 as _, - stream_key: self.logical_pk().iter().map(|x| *x as u32).collect(), + stream_key: self.stream_key().iter().map(|x| *x as u32).collect(), fields: self.schema().to_prost(), append_only: self.append_only(), } diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index 41dd857282fbe..1ca7f513cb3ab 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -35,8 +35,8 @@ pub struct PlanBase { #[educe(Hash(ignore))] pub ctx: OptimizerContextRef, pub schema: Schema, - /// the pk indices of the PlanNode's output, a empty logical_pk vec means there is no pk - pub logical_pk: Vec, + /// the pk indices of the PlanNode's output, a empty stream key vec means there is no stream key + pub stream_key: Vec, /// The order property of the PlanNode's output, store an `&Order::any()` here will not affect /// correctness, but insert unnecessary sort in plan pub order: Order, @@ -59,8 +59,8 @@ impl generic::GenericPlanRef for PlanBase { &self.schema } - fn logical_pk(&self) -> &[usize] { - &self.logical_pk + fn stream_key(&self) -> &[usize] { + &self.stream_key } fn ctx(&self) -> OptimizerContextRef { @@ -94,7 +94,7 @@ impl PlanBase { pub fn new_logical( ctx: OptimizerContextRef, schema: Schema, - logical_pk: Vec, + stream_key: Vec, functional_dependency: FunctionalDependencySet, ) -> Self { let id = ctx.next_plan_node_id(); @@ -103,7 +103,7 @@ impl PlanBase { id, ctx, schema, - logical_pk, + stream_key, dist: Distribution::Single, order: Order::any(), // Logical plan node won't touch `append_only` field @@ -118,7 +118,7 @@ impl PlanBase { Self::new_logical( node.ctx(), node.schema(), - node.logical_pk().unwrap_or_default(), + node.stream_key().unwrap_or_default(), node.functional_dependency(), ) } @@ -133,7 +133,7 @@ impl PlanBase { Self::new_stream( logical.ctx(), logical.schema(), - logical.logical_pk().unwrap_or_default().to_vec(), + logical.stream_key().unwrap_or_default().to_vec(), logical.functional_dependency(), dist, append_only, @@ -145,7 +145,7 @@ impl PlanBase { pub fn new_stream( ctx: OptimizerContextRef, schema: Schema, - logical_pk: Vec, + stream_key: Vec, functional_dependency: FunctionalDependencySet, dist: Distribution, append_only: bool, @@ -160,7 +160,7 @@ impl PlanBase { schema, dist, order: Order::any(), - logical_pk, + stream_key, append_only, emit_on_window_close, functional_dependency, @@ -191,7 +191,7 @@ impl PlanBase { schema, dist, order, - logical_pk: vec![], + stream_key: vec![], // Batch plan node won't touch `append_only` field append_only: true, emit_on_window_close: false, // TODO(rc): batch EOWC support? @@ -204,7 +204,7 @@ impl PlanBase { PlanBase::new_stream( plan_node.ctx(), plan_node.schema().clone(), - plan_node.logical_pk().to_vec(), + plan_node.stream_key().to_vec(), plan_node.functional_dependency().clone(), plan_node.distribution().clone(), plan_node.append_only(), @@ -233,8 +233,8 @@ macro_rules! impl_base_delegate { pub fn schema(&self) -> &Schema { &self.plan_base().schema } - pub fn logical_pk(&self) -> &[usize] { - &self.plan_base().logical_pk + pub fn stream_key(&self) -> &[usize] { + &self.plan_base().stream_key } pub fn order(&self) -> &Order { &self.plan_base().order diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 828f509351b37..598318ed22d0c 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -76,7 +76,7 @@ pub trait StreamPlanNode: GenericPlanNode { id: ctx.next_plan_node_id(), ctx, schema: self.schema(), - logical_pk: self.logical_pk().unwrap_or_default(), + stream_key: self.stream_key().unwrap_or_default(), dist: self.distribution(), append_only: self.append_only(), emit_on_window_close: self.emit_on_window_close(), @@ -95,8 +95,8 @@ impl generic::GenericPlanRef for PlanRef { &self.0.schema } - fn logical_pk(&self) -> &[usize] { - &self.0.logical_pk + fn stream_key(&self) -> &[usize] { + &self.0.stream_key } fn ctx(&self) -> OptimizerContextRef { @@ -113,8 +113,8 @@ impl generic::GenericPlanRef for PlanBase { &self.schema } - fn logical_pk(&self) -> &[usize] { - &self.logical_pk + fn stream_key(&self) -> &[usize] { + &self.stream_key } fn ctx(&self) -> OptimizerContextRef { @@ -266,7 +266,7 @@ impl HashJoin { // dedup the pk in dist key.. let mut deduped_input_pk_indices = vec![]; - for input_pk_idx in input.logical_pk() { + for input_pk_idx in input.stream_key() { if !pk_indices.contains(input_pk_idx) && !deduped_input_pk_indices.contains(input_pk_idx) { @@ -410,7 +410,7 @@ pub struct PlanBase { #[educe(Hash(ignore))] pub ctx: OptimizerContextRef, pub schema: Schema, - pub logical_pk: Vec, + pub stream_key: Vec, #[educe(PartialEq(ignore))] #[educe(Hash(ignore))] pub dist: Distribution, @@ -608,7 +608,7 @@ pub fn to_stream_prost_body( .infer_internal_table_catalog( input.schema(), input.ctx(), - input.logical_pk(), + input.stream_key(), me.vnode_col_idx, ) .with_id(state.gen_table_id_wrapped()); @@ -765,7 +765,7 @@ pub fn to_stream_prost_body( me.infer_internal_table_catalog( input.schema(), input.ctx(), - input.logical_pk(), + input.stream_key(), None, ) .with_id(state.gen_table_id_wrapped()) diff --git a/src/frontend/src/optimizer/plan_node/stream_derive.rs b/src/frontend/src/optimizer/plan_node/stream_derive.rs index 404ca04a39fab..f3da2b1b6a1dd 100644 --- a/src/frontend/src/optimizer/plan_node/stream_derive.rs +++ b/src/frontend/src/optimizer/plan_node/stream_derive.rs @@ -25,7 +25,7 @@ impl GenericPlanNode for DynamicFilter { todo!("new plan node derivation") } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { todo!("new plan node derivation") } @@ -56,7 +56,7 @@ impl GenericPlanNode for Exchange { todo!("new plan node derivation") } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { todo!("new plan node derivation") } @@ -88,8 +88,8 @@ impl GenericPlanNode for DeltaJoin { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -120,8 +120,8 @@ impl GenericPlanNode for Expand { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -152,8 +152,8 @@ impl GenericPlanNode for Filter { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -184,8 +184,8 @@ impl GenericPlanNode for SimpleAgg { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -216,8 +216,8 @@ impl GenericPlanNode for GroupTopN { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -248,8 +248,8 @@ impl GenericPlanNode for HashAgg { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -280,8 +280,8 @@ impl GenericPlanNode for HashJoin { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -312,8 +312,8 @@ impl GenericPlanNode for HopWindow { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -344,8 +344,8 @@ impl GenericPlanNode for IndexScan { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -376,8 +376,8 @@ impl GenericPlanNode for StatelessSimpleAgg { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -408,7 +408,7 @@ impl GenericPlanNode for Materialize { todo!("new plan node derivation") } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { todo!("new plan node derivation") } @@ -440,8 +440,8 @@ impl GenericPlanNode for ProjectSet { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -472,8 +472,8 @@ impl GenericPlanNode for Project { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -506,7 +506,7 @@ impl GenericPlanNode for Sink { todo!("new plan node derivation") } - fn logical_pk(&self) -> Option> { + fn stream_key(&self) -> Option> { todo!("new plan node derivation") } @@ -538,8 +538,8 @@ impl GenericPlanNode for Source { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -570,8 +570,8 @@ impl GenericPlanNode for TableScan { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -602,8 +602,8 @@ impl GenericPlanNode for TopN { self.core.schema() } - fn logical_pk(&self) -> Option> { - self.core.logical_pk() + fn stream_key(&self) -> Option> { + self.core.stream_key() } fn ctx(&self) -> OptimizerContextRef { diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs index c576d5e2d83d3..febf550049265 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dml.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dml.rs @@ -33,7 +33,7 @@ impl StreamDml { let base = PlanBase::new_stream( input.ctx(), input.schema().clone(), - input.logical_pk().to_vec(), + input.stream_key().to_vec(), input.functional_dependency().clone(), input.distribution().clone(), append_only, diff --git a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs index dea3d0eb49889..6b13b88a38bf2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs @@ -101,7 +101,7 @@ impl StreamEowcOverWindow { tbl_builder.add_order_column(order_key_index, OrderType::ascending()); order_cols.insert(order_key_index); } - for idx in self.logical.input.logical_pk() { + for idx in self.logical.input.stream_key() { if !order_cols.contains(idx) { tbl_builder.add_order_column(*idx, OrderType::ascending()); order_cols.insert(*idx); diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index b5da1804aee71..b70b127a5522c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -37,7 +37,7 @@ impl StreamExchange { let base = PlanBase::new_stream( input.ctx(), input.schema().clone(), - input.logical_pk().to_vec(), + input.stream_key().to_vec(), input.functional_dependency().clone(), dist, input.append_only(), @@ -53,7 +53,7 @@ impl StreamExchange { pub fn new_no_shuffle(input: PlanRef) -> Self { let ctx = input.ctx(); - let pk_indices = input.logical_pk().to_vec(); + let pk_indices = input.stream_key().to_vec(); // Dispatch executor won't change the append-only behavior of the stream. let base = PlanBase::new_stream( ctx, diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs index 761807cba1ae7..93209aa0b8800 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -93,7 +93,7 @@ impl StreamNode for StreamGroupTopN { .infer_internal_table_catalog( input.schema(), input.ctx(), - input.logical_pk(), + input.stream_key(), self.vnode_col_idx, ) .with_id(state.gen_table_id_wrapped()); diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 66be991bae95c..4b85612678f94 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -140,7 +140,7 @@ impl StreamMaterialize { TableType::MaterializedView => { assert_matches!(user_distributed_by, RequiredDist::Any); // ensure the same pk will not shuffle to different node - RequiredDist::shard_by_key(input.schema().len(), input.logical_pk()) + RequiredDist::shard_by_key(input.schema().len(), input.stream_key()) } TableType::Index => { assert_matches!( @@ -292,7 +292,7 @@ impl PlanTreeNodeUnary for StreamMaterialize { assert_eq!(a.type_name, b.type_name); assert_eq!(a.sub_fields, b.sub_fields); }); - assert_eq!(new.plan_base().logical_pk, self.plan_base().logical_pk); + assert_eq!(new.plan_base().stream_key, self.plan_base().stream_key); new } } diff --git a/src/frontend/src/optimizer/plan_node/stream_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_over_window.rs index b07c75a1f261e..e77145bac748d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs @@ -68,7 +68,7 @@ impl StreamOverWindow { tbl_builder.add_order_column(o.column_index, o.order_type); } } - for &idx in self.logical.input.logical_pk() { + for &idx in self.logical.input.stream_key() { if order_cols.insert(idx) { tbl_builder.add_order_column(idx, OrderType::ascending()); } diff --git a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs index 1562306825dfd..a2a5f834400a5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs @@ -40,7 +40,7 @@ impl StreamRowIdGen { let base = PlanBase::new_stream( input.ctx(), input.schema().clone(), - input.logical_pk().to_vec(), + input.stream_key().to_vec(), input.functional_dependency().clone(), distribution, input.append_only(), diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 93e8030d49566..795cdcbfda06b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -96,7 +96,7 @@ impl StreamShare { identity: self.distill_to_string(), node_body: Some(node_body), operator_id: self.id().0 as _, - stream_key: self.logical_pk().iter().map(|x| *x as u32).collect(), + stream_key: self.stream_key().iter().map(|x| *x as u32).collect(), fields: self.schema().to_prost(), append_only: self.append_only(), }; diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 60e89abf3f5c9..4b9250c4b23f5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -138,7 +138,7 @@ impl StreamSink { } _ => { assert_matches!(user_distributed_by, RequiredDist::Any); - RequiredDist::shard_by_key(input.schema().len(), input.logical_pk()) + RequiredDist::shard_by_key(input.schema().len(), input.stream_key()) } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_sort.rs b/src/frontend/src/optimizer/plan_node/stream_sort.rs index ea130a18af721..9ade7a165500a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sort.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sort.rs @@ -48,7 +48,7 @@ impl StreamEowcSort { assert!(input.watermark_columns().contains(sort_column_index)); let schema = input.schema().clone(); - let logical_pk = input.logical_pk().to_vec(); + let stream_key = input.stream_key().to_vec(); let fd_set = input.functional_dependency().clone(); let dist = input.distribution().clone(); let mut watermark_columns = FixedBitSet::with_capacity(input.schema().len()); @@ -56,7 +56,7 @@ impl StreamEowcSort { let base = PlanBase::new_stream( input.ctx(), schema, - logical_pk, + stream_key, fd_set, dist, true, @@ -92,7 +92,7 @@ impl StreamEowcSort { } } - for idx in self.input.logical_pk() { + for idx in self.input.stream_key() { if !order_cols.contains(idx) { tbl_builder.add_order_column(*idx, OrderType::ascending()); order_cols.insert(*idx); diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 131b58795cbe4..bc841a3efd5cf 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -202,7 +202,7 @@ impl Distill for StreamTableScan { if verbose { let pk = IndicesDisplay { - indices: self.logical_pk(), + indices: self.stream_key(), schema: &self.base.schema, }; vec.push(("pk", pk.distill())); @@ -227,7 +227,7 @@ impl StreamTableScan { pub fn adhoc_to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> PbStreamNode { use risingwave_pb::stream_plan::*; - let stream_key = self.base.logical_pk.iter().map(|x| *x as u32).collect_vec(); + let stream_key = self.base.stream_key.iter().map(|x| *x as u32).collect_vec(); // The required columns from the table (both scan and upstream). let upstream_column_ids = match self.chain_type { diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index 83ab3820bd376..f9be4149ea06b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -43,7 +43,7 @@ impl StreamTopN { let mut base = PlanBase::new_stream_with_logical(&logical, dist, false, false, watermark_columns); if let Some(stream_key) = stream_key { - base.logical_pk = stream_key; + base.stream_key = stream_key; } StreamTopN { base, logical } } @@ -86,7 +86,7 @@ impl PlanTreeNodeUnary for StreamTopN { fn clone_with_input(&self, input: PlanRef) -> Self { let mut logical = self.logical.clone(); logical.input = input; - Self::new_inner(logical, Some(self.logical_pk().to_vec())) + Self::new_inner(logical, Some(self.stream_key().to_vec())) } } @@ -106,7 +106,7 @@ impl StreamNode for StreamTopN { .infer_internal_table_catalog( input.schema(), input.ctx(), - input.logical_pk(), + input.stream_key(), None, ) .with_id(state.gen_table_id_wrapped()) diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index f4ad303cdf2b6..1f764ba37c305 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -40,7 +40,7 @@ impl StreamValues { let base = PlanBase::new_stream( ctx, logical.schema().clone(), - logical.logical_pk().to_vec(), + logical.stream_key().to_vec(), logical.functional_dependency().clone(), Distribution::Single, true, diff --git a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs index 0ca939a207926..2897d67ec5dea 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -43,7 +43,7 @@ impl StreamWatermarkFilter { let base = PlanBase::new_stream( input.ctx(), input.schema().clone(), - input.logical_pk().to_vec(), + input.stream_key().to_vec(), input.functional_dependency().clone(), input.distribution().clone(), input.append_only(), diff --git a/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs b/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs index e563dcf2b41c3..e32385be59d7b 100644 --- a/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs @@ -36,7 +36,7 @@ impl CardinalityVisitor { input_card: Cardinality, eq_set: HashSet, ) -> Cardinality { - let mut unique_keys: Vec> = vec![input.logical_pk().iter().copied().collect()]; + let mut unique_keys: Vec> = vec![input.stream_key().iter().copied().collect()]; // We don't have UNIQUE key now. So we hack here to support some complex queries on // system tables. From d0484ebfe932380eafc97d6d32c91d9d95e77dbb Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Mon, 25 Sep 2023 21:57:36 +0800 Subject: [PATCH 05/17] refactor(frontend): don't expose key range in rw_hummock_*_version (#12522) --- .../rw_catalog/rw_hummock_version.rs | 21 ++++++++++++++++--- 1 file changed, 18 insertions(+), 3 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 550636a6c9c14..f8096b6c3ccbf 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 @@ -48,15 +48,31 @@ pub const RW_HUMMOCK_CHECKPOINT_VERSION: BuiltinTable = BuiltinTable { impl SysCatalogReaderImpl { pub async fn read_hummock_current_version(&self) -> Result> { let version = self.meta_client.get_hummock_current_version().await?; - Ok(version_to_rows(&version)) + Ok(version_to_rows(&remove_key_range_from_version(version))) } pub async fn read_hummock_checkpoint_version(&self) -> Result> { let version = self.meta_client.get_hummock_checkpoint_version().await?; - Ok(version_to_rows(&version)) + Ok(version_to_rows(&remove_key_range_from_version(version))) } } +fn remove_key_range_from_version(mut version: HummockVersion) -> HummockVersion { + // Because key range is too verbose for manual analysis, just don't expose it. + for cg in version.levels.values_mut() { + for level in cg + .levels + .iter_mut() + .chain(cg.l0.as_mut().unwrap().sub_levels.iter_mut()) + { + for sst in &mut level.table_infos { + sst.key_range.take(); + } + } + } + version +} + fn version_to_rows(version: &HummockVersion) -> Vec { version .levels @@ -66,7 +82,6 @@ fn version_to_rows(version: &HummockVersion) -> Vec { Some(ScalarImpl::Int64(version.id as _)), Some(ScalarImpl::Int64(version.max_committed_epoch as _)), Some(ScalarImpl::Int64(version.safe_epoch as _)), - // FIXME #8612: The byte array key_range is encoded to a string by serde_json. We need disable this behavior as it makes it harder to understand the key range. Some(ScalarImpl::Jsonb(json!(cg).into())), ]) }) From 31aa925988feba274d4335b1aa0b714320569774 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Sep 2023 01:13:11 +0800 Subject: [PATCH 06/17] fix(sink): try poll response stream error while sending request (#12525) --- .../connector/SinkWriterStreamObserver.java | 14 +++++-- src/rpc_client/src/lib.rs | 38 ++++++++++++++----- 2 files changed, 38 insertions(+), 14 deletions(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkWriterStreamObserver.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkWriterStreamObserver.java index 5caa9e3533e54..1323133519165 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkWriterStreamObserver.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkWriterStreamObserver.java @@ -38,6 +38,8 @@ public class SinkWriterStreamObserver private TableSchema tableSchema; + private boolean finished = false; + private boolean epochStarted; private long currentEpoch; private Long currentBatchId; @@ -58,6 +60,9 @@ public SinkWriterStreamObserver( @Override public void onNext(ConnectorServiceProto.SinkWriterStreamRequest sinkTask) { + if (finished) { + throw new RuntimeException("unexpected onNext call on a finished writer stream"); + } try { if (sinkTask.hasStart()) { if (isInitialized()) { @@ -169,26 +174,27 @@ public void onNext(ConnectorServiceProto.SinkWriterStreamRequest sinkTask) { throw INVALID_ARGUMENT.withDescription("invalid sink task").asRuntimeException(); } } catch (Exception e) { - LOG.error("sink task error: ", e); + LOG.error("sink writer error: ", e); + cleanup(); responseObserver.onError(e); } } @Override public void onError(Throwable throwable) { - LOG.error("sink task error: ", throwable); + LOG.error("sink writer finishes with error: ", throwable); cleanup(); - responseObserver.onError(throwable); } @Override public void onCompleted() { - LOG.debug("sink task completed"); + LOG.info("sink writer completed"); cleanup(); responseObserver.onCompleted(); } private void cleanup() { + finished = true; if (sink != null) { sink.drop(); } diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index 2d8017e87aa07..6bbcbd2ebf3e2 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -31,12 +31,13 @@ use std::any::type_name; use std::fmt::{Debug, Formatter}; use std::future::Future; use std::iter::repeat; +use std::pin::pin; use std::sync::Arc; use anyhow::anyhow; use async_trait::async_trait; -use futures::future::try_join_all; -use futures::stream::BoxStream; +use futures::future::{select, try_join_all, Either}; +use futures::stream::{BoxStream, Peekable}; use futures::{Stream, StreamExt}; use moka::future::Cache; use rand::prelude::SliceRandom; @@ -58,6 +59,8 @@ mod sink_coordinate_client; mod stream_client; mod tracing; +use std::pin::Pin; + pub use compactor_client::{CompactorClient, GrpcCompactorProxyClient}; pub use compute_client::{ComputeClient, ComputeClientPool, ComputeClientPoolRef}; pub use connector_client::{ConnectorClient, SinkCoordinatorStreamHandle, SinkWriterStreamHandle}; @@ -173,7 +176,7 @@ macro_rules! meta_rpc_client_method_impl { pub struct BidiStreamHandle { request_sender: Sender, - response_stream: BoxStream<'static, std::result::Result>, + response_stream: Peekable>>, } impl Debug for BidiStreamHandle { @@ -189,7 +192,7 @@ impl BidiStreamHandle { ) -> Self { Self { request_sender, - response_stream, + response_stream: response_stream.peekable(), } } @@ -223,7 +226,7 @@ impl BidiStreamHandle { Ok(( Self { request_sender, - response_stream: response_stream.boxed(), + response_stream: response_stream.boxed().peekable(), }, first_response, )) @@ -238,10 +241,25 @@ impl BidiStreamHandle { } pub async fn send_request(&mut self, request: REQ) -> Result<()> { - Ok(self - .request_sender - .send(request) - .await - .map_err(|_| anyhow!("unable to send request {}", type_name::()))?) + // Poll the response stream to early see the error + let send_request_result = match select( + pin!(self.request_sender.send(request)), + pin!(Pin::new(&mut self.response_stream).peek()), + ) + .await + { + Either::Left((result, _)) => result, + Either::Right((response_result, send_future)) => match response_result { + None => { + return Err(anyhow!("end of response stream").into()); + } + Some(Err(e)) => { + return Err(e.clone().into()); + } + Some(Ok(_)) => send_future.await, + }, + }; + send_request_result + .map_err(|_| anyhow!("unable to send request {}", type_name::()).into()) } } From ea9b94c29adb94145746b79341749375132fdcd9 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Tue, 26 Sep 2023 09:59:19 +0800 Subject: [PATCH 07/17] feat(serverless compaction): add retry strategy for grpc proxy client (#12506) --- src/rpc_client/src/compactor_client.rs | 111 +++++++++++++++++++++---- src/storage/compactor/src/server.rs | 7 +- 2 files changed, 99 insertions(+), 19 deletions(-) diff --git a/src/rpc_client/src/compactor_client.rs b/src/rpc_client/src/compactor_client.rs index 8574e76695740..77fd3e0a44700 100644 --- a/src/rpc_client/src/compactor_client.rs +++ b/src/rpc_client/src/compactor_client.rs @@ -27,10 +27,17 @@ use risingwave_pb::meta::{GetSystemParamsRequest, GetSystemParamsResponse}; use risingwave_pb::monitor_service::monitor_service_client::MonitorServiceClient; use risingwave_pb::monitor_service::{StackTraceRequest, StackTraceResponse}; use tokio::sync::RwLock; +use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tonic::transport::{Channel, Endpoint}; use crate::error::Result; +use crate::retry_rpc; +const ENDPOINT_KEEP_ALIVE_INTERVAL_SEC: u64 = 60; +const ENDPOINT_KEEP_ALIVE_TIMEOUT_SEC: u64 = 60; +const DEFAULT_RETRY_INTERVAL: u64 = 20; +const DEFAULT_RETRY_MAX_DELAY: Duration = Duration::from_secs(5); +const DEFAULT_RETRY_MAX_ATTEMPTS: usize = 3; #[derive(Clone)] pub struct CompactorClient { pub monitor_client: MonitorServiceClient, @@ -82,56 +89,128 @@ impl GrpcCompactorProxyClientCore { #[derive(Debug, Clone)] pub struct GrpcCompactorProxyClient { pub core: Arc>, + endpoint: String, } impl GrpcCompactorProxyClient { - pub fn new(channel: Channel) -> Self { + pub fn new(channel: Channel, endpoint: String) -> Self { let core = Arc::new(RwLock::new(GrpcCompactorProxyClientCore::new(channel))); - Self { core } + Self { core, endpoint } } - fn _recreate_core(&self, _channel: Channel) { - todo!() + async fn recreate_core(&self) { + tracing::info!("GrpcCompactorProxyClient rpc transfer failed, try to reconnect"); + let channel = self.connect_to_endpoint().await; + let mut core = self.core.write().await; + *core = GrpcCompactorProxyClientCore::new(channel); + } + + async fn connect_to_endpoint(&self) -> Channel { + let endpoint = + Endpoint::from_shared(self.endpoint.clone()).expect("Fail to construct tonic Endpoint"); + endpoint + .http2_keep_alive_interval(Duration::from_secs(ENDPOINT_KEEP_ALIVE_INTERVAL_SEC)) + .keep_alive_timeout(Duration::from_secs(ENDPOINT_KEEP_ALIVE_TIMEOUT_SEC)) + .connect_timeout(Duration::from_secs(5)) + .connect() + .await + .expect("Failed to create channel via proxy rpc endpoint.") } pub async fn get_new_sst_ids( &self, request: GetNewSstIdsRequest, ) -> std::result::Result, tonic::Status> { - let mut hummock_client = self.core.read().await.hummock_client.clone(); - hummock_client.get_new_sst_ids(request).await + retry_rpc!(self, get_new_sst_ids, request, GetNewSstIdsResponse) } pub async fn report_compaction_task( &self, request: ReportCompactionTaskRequest, ) -> std::result::Result, tonic::Status> { - let mut hummock_client = self.core.read().await.hummock_client.clone(); - hummock_client.report_compaction_task(request).await + retry_rpc!( + self, + report_compaction_task, + request, + ReportCompactionTaskResponse + ) } pub async fn report_full_scan_task( &self, request: ReportFullScanTaskRequest, ) -> std::result::Result, tonic::Status> { - let mut hummock_client = self.core.read().await.hummock_client.clone(); - hummock_client.report_full_scan_task(request).await + retry_rpc!( + self, + report_full_scan_task, + request, + ReportFullScanTaskResponse + ) } pub async fn report_vacuum_task( &self, request: ReportVacuumTaskRequest, ) -> std::result::Result, tonic::Status> { - let mut hummock_client = self.core.read().await.hummock_client.clone(); - hummock_client.report_vacuum_task(request).await + retry_rpc!(self, report_vacuum_task, request, ReportVacuumTaskResponse) } pub async fn get_system_params( &self, ) -> std::result::Result, tonic::Status> { - let mut system_params_client = self.core.read().await.system_params_client.clone(); - system_params_client - .get_system_params(GetSystemParamsRequest {}) - .await + tokio_retry::RetryIf::spawn( + Self::get_retry_strategy(), + || async { + let mut system_params_client = self.core.read().await.system_params_client.clone(); + let rpc_res = system_params_client + .get_system_params(GetSystemParamsRequest {}) + .await; + if rpc_res.is_err() { + self.recreate_core().await; + } + rpc_res + }, + Self::should_retry, + ) + .await } + + #[inline(always)] + fn get_retry_strategy() -> impl Iterator { + ExponentialBackoff::from_millis(DEFAULT_RETRY_INTERVAL) + .max_delay(DEFAULT_RETRY_MAX_DELAY) + .take(DEFAULT_RETRY_MAX_ATTEMPTS) + .map(jitter) + } + + #[inline(always)] + fn should_retry(status: &tonic::Status) -> bool { + if status.code() == tonic::Code::Unavailable + || status.code() == tonic::Code::Unknown + || (status.code() == tonic::Code::Unauthenticated + && status.message().contains("invalid auth token")) + { + return true; + } + false + } +} + +#[macro_export] +macro_rules! retry_rpc { + ($self:expr, $rpc_call:ident, $request:expr, $response:ty) => { + tokio_retry::RetryIf::spawn( + Self::get_retry_strategy(), + || async { + let mut hummock_client = $self.core.read().await.hummock_client.clone(); + let rpc_res = hummock_client.$rpc_call($request.clone()).await; + if rpc_res.is_err() { + $self.recreate_core().await; + } + rpc_res + }, + Self::should_retry, + ) + .await + }; } diff --git a/src/storage/compactor/src/server.rs b/src/storage/compactor/src/server.rs index e099ed16ffcbd..779eec208f72c 100644 --- a/src/storage/compactor/src/server.rs +++ b/src/storage/compactor/src/server.rs @@ -319,8 +319,9 @@ pub async fn shared_compactor_serve( ); info!("> version: {} ({})", RW_VERSION, GIT_SHA); - let endpoint: &'static str = Box::leak(opts.proxy_rpc_endpoint.clone().into_boxed_str()); - let endpoint = Endpoint::from_static(endpoint); + let endpoint_str = opts.proxy_rpc_endpoint.clone().to_string(); + let endpoint = + Endpoint::from_shared(opts.proxy_rpc_endpoint).expect("Fail to construct tonic Endpoint"); let channel = endpoint .http2_keep_alive_interval(Duration::from_secs(ENDPOINT_KEEP_ALIVE_INTERVAL_SEC)) .keep_alive_timeout(Duration::from_secs(ENDPOINT_KEEP_ALIVE_TIMEOUT_SEC)) @@ -328,7 +329,7 @@ pub async fn shared_compactor_serve( .connect() .await .expect("Failed to create channel via proxy rpc endpoint."); - let grpc_proxy_client = GrpcCompactorProxyClient::new(channel); + let grpc_proxy_client = GrpcCompactorProxyClient::new(channel, endpoint_str); let system_params_response = grpc_proxy_client .get_system_params() .await From f3793716c4be25d5392643fe02637f362389ea7d Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Tue, 26 Sep 2023 11:05:51 +0800 Subject: [PATCH 08/17] fix(metric): fix metric for branched sst info (#12477) --- src/meta/src/hummock/metrics_utils.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/meta/src/hummock/metrics_utils.rs b/src/meta/src/hummock/metrics_utils.rs index 8bcc1f1d2c1c6..9917d559a0709 100644 --- a/src/meta/src/hummock/metrics_utils.rs +++ b/src/meta/src/hummock/metrics_utils.rs @@ -483,15 +483,15 @@ pub fn trigger_split_stat( .with_label_values(&[&group_label]) .set(member_table_id_len as _); - let branched_sst_count = branched_ssts + let branched_sst_count: usize = branched_ssts .values() - .map(|branched_map| branched_map.iter()) - .flat_map(|branched_map| { + .map(|branched_map| { branched_map - .filter(|(group_id, _sst_id)| **group_id == compaction_group_id) - .map(|(_, v)| v) + .keys() + .filter(|group_id| **group_id == compaction_group_id) + .count() }) - .sum::(); + .sum(); metrics .branched_sst_count From 42290e4944e73be96a411abf0b84fe0a3299d79e Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Tue, 26 Sep 2023 12:40:55 +0800 Subject: [PATCH 09/17] feat(sink): Support clickhouse sink if rw columns is subset of ck (#12123) --- src/connector/src/sink/clickhouse.rs | 74 +++++++++++++++++++++------- 1 file changed, 55 insertions(+), 19 deletions(-) diff --git a/src/connector/src/sink/clickhouse.rs b/src/connector/src/sink/clickhouse.rs index 1e2e521066600..ef4cc8b993a9b 100644 --- a/src/connector/src/sink/clickhouse.rs +++ b/src/connector/src/sink/clickhouse.rs @@ -13,7 +13,7 @@ // limitations under the License. use core::fmt::Debug; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use anyhow::anyhow; use clickhouse::{Client, Row as ClickHouseRow}; @@ -22,7 +22,6 @@ use risingwave_common::array::{Op, RowRef, StreamChunk}; use risingwave_common::catalog::Schema; use risingwave_common::row::Row; use risingwave_common::types::{DataType, ScalarRefImpl, Serial}; -use risingwave_common::util::iter_util::ZipEqFast; use serde::ser::{SerializeSeq, SerializeStruct}; use serde::Serialize; use serde_derive::Deserialize; @@ -87,24 +86,57 @@ impl ClickHouseSink { } /// Check that the column names and types of risingwave and clickhouse are identical - fn check_column_name_and_type(&self, clickhouse_column: Vec) -> Result<()> { - let ck_fields_name = build_fields_name_type_from_schema(&self.schema)?; - if !ck_fields_name.len().eq(&clickhouse_column.len()) { - return Err(SinkError::ClickHouse("Schema len not match".to_string())); + fn check_column_name_and_type(&self, clickhouse_columns_desc: &[SystemColumn]) -> Result<()> { + let rw_fields_name = build_fields_name_type_from_schema(&self.schema)?; + let clickhouse_columns_desc: HashMap = clickhouse_columns_desc + .iter() + .map(|s| (s.name.clone(), s.clone())) + .collect(); + + if rw_fields_name.len().gt(&clickhouse_columns_desc.len()) { + return Err(SinkError::ClickHouse("The nums of the RisingWave column must be greater than/equal to the length of the Clickhouse column".to_string())); } - ck_fields_name - .iter() - .zip_eq_fast(clickhouse_column) - .try_for_each(|(key, value)| { - if !key.0.eq(&value.name) { - return Err(SinkError::ClickHouse(format!( - "Column name is not match, risingwave is {:?} and clickhouse is {:?}", - key.0, value.name - ))); - } - Self::check_and_correct_column_type(&key.1, &value) + for i in rw_fields_name { + let value = clickhouse_columns_desc.get(&i.0).ok_or_else(|| { + SinkError::ClickHouse(format!( + "Column name don't find in clickhouse, risingwave is {:?} ", + i.0 + )) })?; + + Self::check_and_correct_column_type(&i.1, value)?; + } + Ok(()) + } + + /// Check that the column names and types of risingwave and clickhouse are identical + fn check_pk_match(&self, clickhouse_columns_desc: &[SystemColumn]) -> Result<()> { + let mut clickhouse_pks: HashSet = clickhouse_columns_desc + .iter() + .filter(|s| s.is_in_primary_key == 1) + .map(|s| s.name.clone()) + .collect(); + + for (_, field) in self + .schema + .fields() + .iter() + .enumerate() + .filter(|(index, _)| self.pk_indices.contains(index)) + { + if !clickhouse_pks.remove(&field.name) { + return Err(SinkError::ClickHouse( + "Clicklhouse and RisingWave pk is not match".to_string(), + )); + } + } + + if !clickhouse_pks.is_empty() { + return Err(SinkError::ClickHouse( + "Clicklhouse and RisingWave pk is not match".to_string(), + )); + } Ok(()) } @@ -201,7 +233,10 @@ impl Sink for ClickHouseSink { self.config.common.database, self.config.common.table ))); } - self.check_column_name_and_type(clickhouse_column)?; + self.check_column_name_and_type(&clickhouse_column)?; + if !self.is_append_only { + self.check_pk_match(&clickhouse_column)?; + } Ok(()) } @@ -443,10 +478,11 @@ impl SinkWriter for ClickHouseSinkWriter { } } -#[derive(ClickHouseRow, Deserialize)] +#[derive(ClickHouseRow, Deserialize, Clone)] struct SystemColumn { name: String, r#type: String, + is_in_primary_key: u8, } /// Serialize this structure to simulate the `struct` call clickhouse interface From b75ff7976d751df683446569e4b9f87f81cfaf71 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Sep 2023 13:58:38 +0800 Subject: [PATCH 10/17] feat(java-binding): generate jni method signature with macro (#12487) --- Cargo.lock | 3 + src/java_binding/Cargo.toml | 1 + src/java_binding/make-java-binding.toml | 4 +- src/java_binding/src/lib.rs | 16 +- src/jni_core/Cargo.toml | 2 + src/jni_core/src/jvm_runtime.rs | 201 ++----------- src/jni_core/src/lib.rs | 7 +- src/jni_core/src/macros.rs | 358 ++++++++++++++++++++++++ 8 files changed, 412 insertions(+), 180 deletions(-) create mode 100644 src/jni_core/src/macros.rs diff --git a/Cargo.lock b/Cargo.lock index a7caa3148b99c..42a08e60f43d5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7194,6 +7194,7 @@ dependencies = [ name = "risingwave_java_binding" version = "0.1.0" dependencies = [ + "jni", "prost", "risingwave_common", "risingwave_expr", @@ -7208,10 +7209,12 @@ name = "risingwave_jni_core" version = "0.1.0" dependencies = [ "bytes", + "cfg-or-panic", "futures", "itertools 0.11.0", "jni", "madsim-tokio", + "paste", "prost", "risingwave_common", "risingwave_expr", diff --git a/src/java_binding/Cargo.toml b/src/java_binding/Cargo.toml index d8d90693f44a6..2da34aa53af77 100644 --- a/src/java_binding/Cargo.toml +++ b/src/java_binding/Cargo.toml @@ -10,6 +10,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] +jni = "0.21.1" prost = "0.11" risingwave_common = { workspace = true } risingwave_jni_core = { workspace = true } diff --git a/src/java_binding/make-java-binding.toml b/src/java_binding/make-java-binding.toml index 3be65ec2158a6..957cec9c762f5 100644 --- a/src/java_binding/make-java-binding.toml +++ b/src/java_binding/make-java-binding.toml @@ -15,7 +15,7 @@ script = ''' #!/usr/bin/env bash set -ex cd java -mvn install --no-transfer-progress --pl java-binding-integration-test --am -DskipTests=true +mvn install --no-transfer-progress --pl java-binding-integration-test --am -DskipTests=true -Dmaven.javadoc.skip=true mvn dependency:copy-dependencies --no-transfer-progress --pl java-binding-integration-test ''' @@ -109,7 +109,7 @@ RISINGWAVE_ROOT=$(git rev-parse --show-toplevel) cd ${RISINGWAVE_ROOT}/java -mvn install --pl java-binding-benchmark --am -DskipTests=true +mvn install --pl java-binding-benchmark --am -DskipTests=true -Dmaven.javadoc.skip=true mvn dependency:copy-dependencies --pl java-binding-benchmark diff --git a/src/java_binding/src/lib.rs b/src/java_binding/src/lib.rs index aa7e564ed1ace..6edf4d29ce557 100644 --- a/src/java_binding/src/lib.rs +++ b/src/java_binding/src/lib.rs @@ -12,4 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub use risingwave_jni_core::*; +#![feature(result_option_inspect)] + +use std::ffi::c_void; + +use jni::sys::{jint, JNI_VERSION_1_2}; +use jni::JavaVM; +use risingwave_jni_core::register_native_method_for_jvm; + +#[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")); + JNI_VERSION_1_2 +} diff --git a/src/jni_core/Cargo.toml b/src/jni_core/Cargo.toml index c8bba371c8dea..bc70ef8a73114 100644 --- a/src/jni_core/Cargo.toml +++ b/src/jni_core/Cargo.toml @@ -11,9 +11,11 @@ normal = ["workspace-hack"] [dependencies] bytes = "1" +cfg-or-panic = "0.2" futures = { version = "0.3", default-features = false, features = ["alloc"] } itertools = "0.11" jni = "0.21.1" +paste = "1" prost = "0.11" risingwave_common = { workspace = true } risingwave_hummock_sdk = { workspace = true } diff --git a/src/jni_core/src/jvm_runtime.rs b/src/jni_core/src/jvm_runtime.rs index 5559abd3ffa3f..62630528961dc 100644 --- a/src/jni_core/src/jvm_runtime.rs +++ b/src/jni_core/src/jvm_runtime.rs @@ -91,12 +91,12 @@ pub static JVM: LazyLock> = LazyLock::new(|| { tracing::info!("initialize JVM successfully"); - register_native_method_for_jvm(&jvm); + register_native_method_for_jvm(&jvm).unwrap(); Ok(jvm) }); -fn register_native_method_for_jvm(jvm: &JavaVM) { +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!("jvm attach thread error: {:?}", e)) @@ -106,179 +106,30 @@ fn register_native_method_for_jvm(jvm: &JavaVM) { .find_class("com/risingwave/java/binding/Binding") .inspect_err(|e| tracing::error!("jvm find class error: {:?}", e)) .unwrap(); - env.register_native_methods( - binding_class, - &[ - NativeMethod { - name: JNIString::from("vnodeCount"), - sig: JNIString::from("()I"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_vnodeCount as *mut c_void, - }, - #[cfg(not(madsim))] - NativeMethod { - name: JNIString::from("hummockIteratorNew"), - sig: JNIString::from("([B)J"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_hummockIteratorNew - as *mut c_void, - }, - #[cfg(not(madsim))] - NativeMethod { - name: JNIString::from("hummockIteratorNext"), - sig: JNIString::from("(J)J"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_hummockIteratorNext - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("hummockIteratorClose"), - sig: JNIString::from("(J)V"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_hummockIteratorClose - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetKey"), - sig: JNIString::from("(J)[B"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetKey as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetOp"), - sig: JNIString::from("(J)I"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetOp as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowIsNull"), - sig: JNIString::from("(JI)Z"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowIsNull as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetInt16Value"), - sig: JNIString::from("(JI)S"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetInt16Value - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetInt32Value"), - sig: JNIString::from("(JI)I"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetInt32Value - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetInt64Value"), - sig: JNIString::from("(JI)J"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetInt64Value - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetFloatValue"), - sig: JNIString::from("(JI)F"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetFloatValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetDoubleValue"), - sig: JNIString::from("(JI)D"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetDoubleValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetBooleanValue"), - sig: JNIString::from("(JI)Z"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetBooleanValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetStringValue"), - sig: JNIString::from("(JI)Ljava/lang/String;"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetStringValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetTimestampValue"), - sig: JNIString::from("(JI)Ljava/sql/Timestamp;"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetTimestampValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetDecimalValue"), - sig: JNIString::from("(JI)Ljava/math/BigDecimal;"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetDecimalValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetTimeValue"), - sig: JNIString::from("(JI)Ljava/sql/Time;"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetTimeValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetDateValue"), - sig: JNIString::from("(JI)Ljava/sql/Date;"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetDateValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetIntervalValue"), - sig: JNIString::from("(JI)Ljava/lang/String;"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetIntervalValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetJsonbValue"), - sig: JNIString::from("(JI)Ljava/lang/String;"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetJsonbValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetByteaValue"), - sig: JNIString::from("(JI)[B"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetByteaValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowGetArrayValue"), - sig: JNIString::from("(JILjava/lang/Class;)Ljava/lang/Object;"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowGetArrayValue - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("rowClose"), - sig: JNIString::from("(J)V"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_rowClose as *mut c_void, - }, - NativeMethod { - name: JNIString::from("streamChunkIteratorNew"), - sig: JNIString::from("([B)J"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_streamChunkIteratorNew - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("streamChunkIteratorNext"), - sig: JNIString::from("(J)J"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_streamChunkIteratorNext - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("streamChunkIteratorClose"), - sig: JNIString::from("(J)V"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_streamChunkIteratorClose - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("streamChunkIteratorFromPretty"), - sig: JNIString::from("(Ljava/lang/String;)J"), - fn_ptr: - crate::Java_com_risingwave_java_binding_Binding_streamChunkIteratorFromPretty - as *mut c_void, - }, - NativeMethod { - name: JNIString::from("sendCdcSourceMsgToChannel"), - sig: JNIString::from("(J[B)Z"), - fn_ptr: crate::Java_com_risingwave_java_binding_Binding_sendCdcSourceMsgToChannel - as *mut c_void, - }, - ], - ) - .inspect_err(|e| tracing::error!("jvm register native methods error: {:?}", e)) - .unwrap(); + use crate::*; + macro_rules! gen_native_method_array { + () => {{ + $crate::for_all_native_methods! {gen_native_method_array} + }}; + ({$({ $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, + } + }, + )* + ] + } + } + env.register_native_methods(binding_class, &gen_native_method_array!()) + .inspect_err(|e| tracing::error!("jvm register native methods error: {:?}", e))?; tracing::info!("register native methods for jvm successfully"); + Ok(()) } diff --git a/src/jni_core/src/lib.rs b/src/jni_core/src/lib.rs index 6fa6f2f10e991..9ebce58b426a7 100644 --- a/src/jni_core/src/lib.rs +++ b/src/jni_core/src/lib.rs @@ -20,6 +20,7 @@ pub mod hummock_iterator; pub mod jvm_runtime; +mod macros; pub mod stream_chunk_iterator; use std::backtrace::Backtrace; @@ -28,6 +29,7 @@ use std::ops::{Deref, DerefMut}; use std::slice::from_raw_parts; use std::sync::{Arc, LazyLock, OnceLock}; +use cfg_or_panic::cfg_or_panic; use hummock_iterator::{HummockJavaBindingIterator, KeyedRow}; use jni::objects::{ AutoElements, GlobalRef, JByteArray, JClass, JMethodID, JObject, JStaticMethodID, JString, @@ -51,6 +53,7 @@ use thiserror::Error; use tokio::runtime::Runtime; use tokio::sync::mpsc::Sender; +pub use crate::jvm_runtime::register_native_method_for_jvm; use crate::stream_chunk_iterator::{StreamChunkIterator, StreamChunkRow}; pub type GetEventStreamJniSender = Sender; @@ -296,7 +299,7 @@ pub extern "system" fn Java_com_risingwave_java_binding_Binding_vnodeCount( VirtualNode::COUNT as jint } -#[cfg(not(madsim))] +#[cfg_or_panic(not(madsim))] #[no_mangle] pub extern "system" fn Java_com_risingwave_java_binding_Binding_hummockIteratorNew<'a>( env: EnvParam<'a>, @@ -309,7 +312,7 @@ pub extern "system" fn Java_com_risingwave_java_binding_Binding_hummockIteratorN }) } -#[cfg(not(madsim))] +#[cfg_or_panic(not(madsim))] #[no_mangle] pub extern "system" fn Java_com_risingwave_java_binding_Binding_hummockIteratorNext<'a>( env: EnvParam<'a>, diff --git a/src/jni_core/src/macros.rs b/src/jni_core/src/macros.rs new file mode 100644 index 0000000000000..bdb5c60ec3f82 --- /dev/null +++ b/src/jni_core/src/macros.rs @@ -0,0 +1,358 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#[macro_export] +macro_rules! gen_class_name { + ($last:ident) => { + stringify! {$last} + }; + ($first:ident . $($rest:ident).+) => { + concat! {stringify! {$first}, "/", gen_class_name! {$($rest).+} } + } +} + +#[macro_export] +macro_rules! gen_jni_sig_inner { + ($(public)? static native $($rest:tt)*) => { + gen_jni_sig_inner! { $($rest)* } + }; + ($($ret:ident).+ $($func_name:ident)? ($($args:tt)*)) => { + concat! {"(", gen_jni_sig_inner!{$($args)*}, ")", gen_jni_sig_inner! {$($ret).+} } + }; + ($($ret:ident).+ [] $($func_name:ident)? ($($args:tt)*)) => { + concat! {"(", gen_jni_sig_inner!{$($args)*}, ")", gen_jni_sig_inner! {$($ret).+ []} } + }; + (boolean) => { + "Z" + }; + (byte) => { + "B" + }; + (char) => { + "C" + }; + (short) => { + "S" + }; + (int) => { + "I" + }; + (long) => { + "J" + }; + (float) => { + "F" + }; + (double) => { + "D" + }; + (void) => { + "V" + }; + (String) => { + gen_jni_sig_inner! { java.lang.String } + }; + (Object) => { + gen_jni_sig_inner! { java.lang.Object } + }; + (Class) => { + gen_jni_sig_inner! { java.lang.Class } + }; + ($($class_part:ident).+) => { + concat! {"L", gen_class_name! {$($class_part).+}, ";"} + }; + ($($class_part:ident).+ $(.)? [] $($param_name:ident)? $(,$($rest:tt)*)?) => { + concat! { "[", gen_jni_sig_inner! {$($class_part).+}, gen_jni_sig_inner! {$($($rest)*)?}} + }; + (Class $(< ? >)? $($param_name:ident)? $(,$($rest:tt)*)?) => { + concat! { gen_jni_sig_inner! { Class }, gen_jni_sig_inner! {$($($rest)*)?}} + }; + ($($class_part:ident).+ $($param_name:ident)? $(,$($rest:tt)*)?) => { + concat! { gen_jni_sig_inner! {$($class_part).+}, gen_jni_sig_inner! {$($($rest)*)?}} + }; + () => { + "" + }; + ($($invalid:tt)*) => { + compile_error!(concat!("unsupported type {{", stringify!($($invalid)*), "}}")) + }; +} + +#[macro_export] +macro_rules! gen_jni_sig { + ($($input:tt)*) => {{ + // this macro only provide with a expression context + gen_jni_sig_inner! {$($input)*} + }} +} + +#[macro_export] +macro_rules! for_all_plain_native_methods { + ($macro:path $(,$args:tt)*) => { + $macro! { + { + public static native int vnodeCount(); + + // hummock iterator method + // Return a pointer to the iterator + static native long hummockIteratorNew(byte[] readPlan); + + // return a pointer to the next row + static native long hummockIteratorNext(long pointer); + + // Since the underlying rust does not have garbage collection, we will have to manually call + // close on the iterator to release the iterator instance pointed by the pointer. + static native void hummockIteratorClose(long pointer); + + // row method + static native byte[] rowGetKey(long pointer); + + static native int rowGetOp(long pointer); + + static native boolean rowIsNull(long pointer, int index); + + static native short rowGetInt16Value(long pointer, int index); + + static native int rowGetInt32Value(long pointer, int index); + + static native long rowGetInt64Value(long pointer, int index); + + static native float rowGetFloatValue(long pointer, int index); + + static native double rowGetDoubleValue(long pointer, int index); + + static native boolean rowGetBooleanValue(long pointer, int index); + + static native String rowGetStringValue(long pointer, int index); + + static native java.sql.Timestamp rowGetTimestampValue(long pointer, int index); + + static native java.math.BigDecimal rowGetDecimalValue(long pointer, int index); + + static native java.sql.Time rowGetTimeValue(long pointer, int index); + + static native java.sql.Date rowGetDateValue(long pointer, int index); + + static native String rowGetIntervalValue(long pointer, int index); + + static native String rowGetJsonbValue(long pointer, int index); + + static native byte[] rowGetByteaValue(long pointer, int index); + + // TODO: object or object array? + static native Object rowGetArrayValue(long pointer, int index, Class clazz); + + // Since the underlying rust does not have garbage collection, we will have to manually call + // close on the row to release the row instance pointed by the pointer. + static native void rowClose(long pointer); + + // stream chunk iterator method + static native long streamChunkIteratorNew(byte[] streamChunkPayload); + + static native long streamChunkIteratorNext(long pointer); + + static native void streamChunkIteratorClose(long pointer); + + static native long streamChunkIteratorFromPretty(String str); + + public static native boolean sendCdcSourceMsgToChannel(long channelPtr, byte[] msg); + } + $(,$args)* + } + }; +} + +#[macro_export] +macro_rules! for_single_native_method { + ( + {$($ret:tt).+ $func_name:ident ($($args:tt)*)}, + $macro:path + $(,$extra_args:tt)* + ) => { + $macro! { + $func_name, + {$($ret).+}, + {$($args)*} + } + }; + ( + {$($ret:tt).+ [] $func_name:ident ($($args:tt)*)}, + $macro:path + $(,$extra_args:tt)* + ) => { + $macro! { + $func_name, + {$($ret).+ []}, + {$($args)*} + } + }; +} + +#[macro_export] +macro_rules! for_all_native_methods { + ( + {$($input:tt)*}, + $macro:path + $(,$extra_args:tt)* + ) => {{ + $crate::for_all_native_methods! { + {$($input)*}, + {}, + $macro + $(,$extra_args)* + } + }}; + ( + { + $(public)? static native $($ret:tt).+ $func_name:ident($($args:tt)*); $($rest:tt)* + }, + { + $({$prev_func_name:ident, {$($prev_ret:tt)*}, {$($prev_args:tt)*}})* + }, + $macro:path + $(,$extra_args:tt)* + ) => { + $crate::for_all_native_methods! { + {$($rest)*}, + { + $({$prev_func_name, {$($prev_ret)*}, {$($prev_args)*}})* + {$func_name, {$($ret).+}, {$($args)*}} + }, + $macro + $(,$extra_args)* + } + }; + ( + { + $(public)? static native $($ret:tt).+ [] $func_name:ident($($args:tt)*); $($rest:tt)* + }, + { + $({$prev_func_name:ident, {$($prev_ret:tt)*}, {$($prev_args:tt)*}})* + }, + $macro:path + $(,$extra_args:tt)* + ) => { + $crate::for_all_native_methods! { + {$($rest)*}, + { + $({$prev_func_name, {$($prev_ret)*}, {$($prev_args)*}})* + {$func_name, {$($ret).+ []}, {$($args)*}} + }, + $macro + $(,$extra_args)* + } + }; + ( + {}, + { + $({$func_name:ident, {$($ret:tt)*}, {$($args:tt)*}})* + }, + $macro:path + $(,$extra_args:tt)* + ) => { + $macro! { + { + $({$func_name, {$($ret)*}, {$($args)*}}),* + } + $(,$extra_args)* + } + }; + ($macro:path $(,$args:tt)*) => {{ + $crate::for_all_plain_native_methods! { + $crate::for_all_native_methods, + $macro + $(,$args)* + } + }}; +} + +#[cfg(test)] +mod tests { + #[test] + fn test_gen_jni_sig() { + assert_eq!(gen_jni_sig!(int), "I"); + assert_eq!(gen_jni_sig!(boolean f(int, short, byte[])), "(IS[B)Z"); + assert_eq!( + gen_jni_sig!(boolean f(int, short, byte[], java.lang.String)), + "(IS[BLjava/lang/String;)Z" + ); + assert_eq!( + gen_jni_sig!(boolean f(int, java.lang.String)), + "(ILjava/lang/String;)Z" + ); + assert_eq!(gen_jni_sig!(public static native int vnodeCount()), "()I"); + assert_eq!( + gen_jni_sig!(long hummockIteratorNew(byte[] readPlan)), + "([B)J" + ); + assert_eq!(gen_jni_sig!(long hummockIteratorNext(long pointer)), "(J)J"); + assert_eq!( + gen_jni_sig!(void hummockIteratorClose(long pointer)), + "(J)V" + ); + assert_eq!(gen_jni_sig!(byte[] rowGetKey(long pointer)), "(J)[B"); + assert_eq!( + gen_jni_sig!(java.sql.Timestamp rowGetTimestampValue(long pointer, int index)), + "(JI)Ljava/sql/Timestamp;" + ); + assert_eq!( + gen_jni_sig!(String rowGetStringValue(long pointer, int index)), + "(JI)Ljava/lang/String;" + ); + assert_eq!( + gen_jni_sig!(static native Object rowGetArrayValue(long pointer, int index, Class clazz)), + "(JILjava/lang/Class;)Ljava/lang/Object;" + ); + } + + #[test] + fn test_for_all_gen() { + macro_rules! gen_array { + (test) => {{ + for_all_native_methods! { + { + public static native int vnodeCount(); + static native long hummockIteratorNew(byte[] readPlan); + public static native byte[] rowGetKey(long pointer); + }, + gen_array + } + }}; + (all) => {{ + for_all_native_methods! { + gen_array + } + }}; + ({$({ $func_name:ident, {$($ret:tt)+}, {$($args:tt)*} }),*}) => {{ + [ + $( + (stringify! {$func_name}, gen_jni_sig! { $($ret)+ ($($args)*)}), + )* + ] + }}; + } + let sig: [(_, _); 3] = gen_array!(test); + assert_eq!( + sig, + [ + ("vnodeCount", "()I"), + ("hummockIteratorNew", "([B)J"), + ("rowGetKey", "(J)[B") + ] + ); + + let sig = gen_array!(all); + assert!(!sig.is_empty()); + } +} From a6fb3412b0b6fe8c0209676be0b3e91559c1c309 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 26 Sep 2023 14:04:51 +0800 Subject: [PATCH 11/17] feat(streaming): recover `no_shuffle_backfill` (#12493) --- ci/scripts/run-backfill-tests.sh | 3 +- ci/scripts/sql/backfill/insert.sql | 2 +- src/meta/src/barrier/progress.rs | 22 +- .../executor/backfill/arrangement_backfill.rs | 2 +- .../executor/backfill/no_shuffle_backfill.rs | 216 +++++++++++------- src/stream/src/executor/backfill/utils.rs | 73 ------ src/stream/src/executor/chain.rs | 4 +- src/stream/src/executor/rearranged_chain.rs | 8 +- src/stream/src/executor/values.rs | 2 +- .../src/task/barrier_manager/managed_state.rs | 6 +- .../src/task/barrier_manager/progress.rs | 17 +- 11 files changed, 178 insertions(+), 177 deletions(-) diff --git a/ci/scripts/run-backfill-tests.sh b/ci/scripts/run-backfill-tests.sh index 6c02442a06255..77a6ef1a5510c 100755 --- a/ci/scripts/run-backfill-tests.sh +++ b/ci/scripts/run-backfill-tests.sh @@ -7,9 +7,8 @@ # Hence keeping it in case we ever need to debug backfill again. # USAGE: -# Start a rw cluster then run this script. # ```sh -# ./risedev d +# cargo make ci-start ci-backfill # ./ci/scripts/run-backfill-tests.sh # ``` diff --git a/ci/scripts/sql/backfill/insert.sql b/ci/scripts/sql/backfill/insert.sql index 18ed763429231..6c12a3e0b897e 100644 --- a/ci/scripts/sql/backfill/insert.sql +++ b/ci/scripts/sql/backfill/insert.sql @@ -2,5 +2,5 @@ insert into t1 SELECT generate_series, '{"orders": {"id": 1, "price": "2.30", "customer_id": 2}}'::jsonb -FROM generate_series(1, 200000); +FROM generate_series(1, 100000); FLUSH; \ No newline at end of file diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index d484e471f4a31..057e0646279dd 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -35,7 +35,7 @@ type ConsumedRows = u64; enum ChainState { Init, ConsumingUpstream(Epoch, ConsumedRows), - Done, + Done(ConsumedRows), } /// Progress of all actors containing chain nodes while creating mview. @@ -93,18 +93,19 @@ impl Progress { match self.states.remove(&actor).unwrap() { ChainState::Init => {} ChainState::ConsumingUpstream(_, old_consumed_rows) => { - if !matches!(new_state, ChainState::Done) { + if !matches!(new_state, ChainState::Done(_)) { self.consumed_rows -= old_consumed_rows; } } - ChainState::Done => panic!("should not report done multiple times"), + ChainState::Done(_) => panic!("should not report done multiple times"), }; match &new_state { ChainState::Init => {} ChainState::ConsumingUpstream(_, new_consumed_rows) => { self.consumed_rows += new_consumed_rows; } - ChainState::Done => { + ChainState::Done(new_consumed_rows) => { + self.consumed_rows += new_consumed_rows; self.done_count += 1; } }; @@ -281,14 +282,21 @@ impl CreateMviewProgressTracker { ) -> Option { let actor = progress.chain_actor_id; let Some(epoch) = self.actor_map.get(&actor).copied() else { - panic!( - "no tracked progress for actor {}, is it already finished?", + // On restart, backfill will ALWAYS notify CreateMviewProgressTracker, + // even if backfill is finished on recovery. + // This is because we don't know if only this actor is finished, + // OR the entire stream job is finished. + // For the first case, we must notify meta. + // For the second case, we can still notify meta, but ignore it here. + tracing::info!( + "no tracked progress for actor {}, the stream job could already be finished", actor ); + return None; }; let new_state = if progress.done { - ChainState::Done + ChainState::Done(progress.consumed_rows) } else { ChainState::ConsumingUpstream(progress.consumed_epoch.into(), progress.consumed_rows) }; diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index d33aed6d6c441..951e4a4235d3d 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -473,7 +473,7 @@ where &mut temporary_state, ).await?; - self.progress.finish(barrier.epoch.curr); + self.progress.finish(barrier.epoch.curr, total_snapshot_processed_rows); yield msg; break; } diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index b6241acdea569..bd40ea8b34e7d 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -21,7 +21,8 @@ use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::catalog::Schema; -use risingwave_common::row::OwnedRow; +use risingwave_common::hash::VnodeBitmapExt; +use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::Datum; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::epoch::EpochPair; @@ -34,8 +35,8 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; use crate::executor::backfill::utils; use crate::executor::backfill::utils::{ - check_all_vnode_finished, compute_bounds, construct_initial_finished_state, get_new_pos, - get_row_count_state, iter_chunks, mapping_chunk, mapping_message, mark_chunk, owned_row_iter, + compute_bounds, construct_initial_finished_state, get_new_pos, iter_chunks, mapping_chunk, + mapping_message, mark_chunk, owned_row_iter, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -44,6 +45,19 @@ use crate::executor::{ }; use crate::task::{ActorId, CreateMviewProgress}; +/// vnode, `is_finished`, `row_count`, all occupy 1 column each. +const METADATA_STATE_LEN: usize = 3; + +/// Schema: | vnode | pk ... | `backfill_finished` | `row_count` | +/// We can decode that into `BackfillState` on recovery. +#[derive(Debug, Eq, PartialEq)] +pub struct BackfillState { + current_pos: Option, + old_state: Option>, + is_finished: bool, + row_count: u64, +} + /// An implementation of the [RFC: Use Backfill To Let Mv On Mv Stream Again](https://github.com/risingwavelabs/rfcs/pull/13). /// `BackfillExecutor` is used to create a materialized view on another materialized view. /// @@ -128,9 +142,7 @@ where // The primary key columns, in the output columns of the upstream_table scan. let pk_in_output_indices = self.upstream_table.pk_in_output_indices().unwrap(); - // schema: | vnode | pk ... | backfill_finished | row_count | - // +1 for vnode, +1 for backfill_finished, +1 for row_count. - let state_len = pk_in_output_indices.len() + 3; + let state_len = pk_in_output_indices.len() + METADATA_STATE_LEN; let pk_order = self.upstream_table.pk_serializer().get_order_types(); @@ -145,16 +157,13 @@ where state_table.init_epoch(first_barrier.epoch); } - let is_finished = if let Some(state_table) = self.state_table.as_ref() { - let is_finished = check_all_vnode_finished(state_table, state_len).await?; - if is_finished { - assert!(!first_barrier.is_newly_added(self.actor_id)); - } - is_finished - } else { - // Maintain backwards compatibility with no state table - !first_barrier.is_newly_added(self.actor_id) - }; + let BackfillState { + mut current_pos, + is_finished, + row_count, + mut old_state, + } = Self::recover_backfill_state(self.state_table.as_ref(), pk_in_output_indices.len()) + .await?; let mut builder = DataChunkBuilder::new(self.upstream_table.schema().data_types(), self.chunk_size); @@ -191,17 +200,9 @@ where // | f | f | t | let to_backfill = !is_finished && !is_snapshot_empty; - // Current position of the upstream_table storage primary key. - // `None` means it starts from the beginning. - let mut current_pos: Option = None; - - // Use these to persist state. - // They contain the backfill position, - // as well as the progress. - // However, they do not contain the vnode key at index 0. - // That is filled in when we flush the state table. + // Use this buffer to construct state, + // which will then be persisted. let mut current_state: Vec = vec![None; state_len]; - let mut old_state: Option> = None; // The first barrier message should be propagated. yield Message::Barrier(first_barrier); @@ -220,13 +221,7 @@ where let mut snapshot_read_epoch = init_epoch; // Keep track of rows from the snapshot. - let mut total_snapshot_processed_rows: u64 = - if let Some(state_table) = self.state_table.as_ref() { - get_row_count_state(state_table, state_len).await? - } else { - // Maintain backwards compatibility with no state_table. - 0 - }; + let mut total_snapshot_processed_rows: u64 = row_count; // Backfill Algorithm: // @@ -415,13 +410,6 @@ where total_snapshot_processed_rows, ); - tracing::trace!( - actor = self.actor_id, - epoch = ?barrier.epoch, - ?current_pos, - total_snapshot_processed_rows, - "Backfill position persisted" - ); // Persist state on barrier Self::persist_state( barrier.epoch, @@ -434,57 +422,73 @@ where ) .await?; + tracing::trace!( + epoch = ?barrier.epoch, + ?current_pos, + total_snapshot_processed_rows, + "Backfill state persisted" + ); + yield Message::Barrier(barrier); // We will switch snapshot at the start of the next iteration of the backfill loop. } } - tracing::trace!( - actor = self.actor_id, - "Backfill has already finished and forward messages directly to the downstream" - ); + tracing::trace!("Backfill has finished, waiting for barrier"); // Wait for first barrier to come after backfill is finished. // So we can update our progress + persist the status. while let Some(Ok(msg)) = upstream.next().await { if let Some(msg) = mapping_message(msg, &self.output_indices) { // If not finished then we need to update state, otherwise no need. - if let Message::Barrier(barrier) = &msg && !is_finished { - // If snapshot was empty, we do not need to backfill, - // but we still need to persist the finished state. - // We currently persist it on the second barrier here rather than first. - // This is because we can't update state table in first epoch, - // since it expects to have been initialized in previous epoch - // (there's no epoch before the first epoch). - if is_snapshot_empty { - current_pos = - Some(construct_initial_finished_state(pk_in_output_indices.len())) - } + if let Message::Barrier(barrier) = &msg { + if is_finished { + // If already finished, no need persist any state. + } else { + // If snapshot was empty, we do not need to backfill, + // but we still need to persist the finished state. + // We currently persist it on the second barrier here rather than first. + // This is because we can't update state table in first epoch, + // since it expects to have been initialized in previous epoch + // (there's no epoch before the first epoch). + if is_snapshot_empty { + current_pos = + Some(construct_initial_finished_state(pk_in_output_indices.len())) + } - // We will update current_pos at least once, - // since snapshot read has to be non-empty, - // Or snapshot was empty and we construct a placeholder state. - debug_assert_ne!(current_pos, None); + // We will update current_pos at least once, + // since snapshot read has to be non-empty, + // Or snapshot was empty and we construct a placeholder state. + debug_assert_ne!(current_pos, None); + + Self::persist_state( + barrier.epoch, + &mut self.state_table, + true, + ¤t_pos, + total_snapshot_processed_rows, + &mut old_state, + &mut current_state, + ) + .await?; + tracing::trace!( + epoch = ?barrier.epoch, + ?current_pos, + total_snapshot_processed_rows, + "Backfill position persisted after completion" + ); + } + // For both backfill finished before recovery, + // and backfill which just finished, we need to update mview tracker, + // it does not persist this information. + self.progress + .finish(barrier.epoch.curr, total_snapshot_processed_rows); tracing::trace!( - actor = self.actor_id, epoch = ?barrier.epoch, - ?current_pos, - total_snapshot_processed_rows, - "Backfill position persisted after completion" + "Updated CreateMaterializedTracker" ); - Self::persist_state( - barrier.epoch, - &mut self.state_table, - true, - ¤t_pos, - total_snapshot_processed_rows, - &mut old_state, - &mut current_state, - ) - .await?; - self.progress.finish(barrier.epoch.curr); yield msg; break; } @@ -492,6 +496,10 @@ where } } + tracing::trace!( + "Backfill has already finished and forward messages directly to the downstream" + ); + // After progress finished + state persisted, // we can forward messages directly to the downstream, // as backfill is finished. @@ -500,14 +508,68 @@ where #[for_await] for msg in upstream { if let Some(msg) = mapping_message(msg?, &self.output_indices) { - if let Some(state_table) = self.state_table.as_mut() && let Message::Barrier(barrier) = &msg { - state_table.commit_no_data_expected(barrier.epoch); - } yield msg; } } } + async fn recover_backfill_state( + state_table: Option<&StateTable>, + pk_len: usize, + ) -> StreamExecutorResult { + let Some(state_table) = state_table else { + // If no state table, but backfill is present, it must be from an old cluster. + // In that case backfill must be finished, otherwise it won't have been persisted. + return Ok(BackfillState { + current_pos: None, + is_finished: true, + row_count: 0, + old_state: None, + }); + }; + let mut vnodes = state_table.vnodes().iter_vnodes_scalar(); + let first_vnode = vnodes.next().unwrap(); + let key: &[Datum] = &[Some(first_vnode.into())]; + let row = state_table.get_row(key).await?; + let expected_state = Self::deserialize_backfill_state(row, pk_len); + + // All vnode partitions should have same state (no scale-in supported). + for vnode in vnodes { + let key: &[Datum] = &[Some(vnode.into())]; + let row = state_table.get_row(key).await?; + let state = Self::deserialize_backfill_state(row, pk_len); + assert_eq!(state, expected_state); + } + Ok(expected_state) + } + + fn deserialize_backfill_state(row: Option, pk_len: usize) -> BackfillState { + let Some(row) = row else { + return BackfillState { + current_pos: None, + is_finished: false, + row_count: 0, + old_state: None, + }; + }; + let row = row.into_inner(); + let mut old_state = vec![None; pk_len + METADATA_STATE_LEN]; + old_state[1..row.len() + 1].clone_from_slice(&row); + let current_pos = Some((&row[0..pk_len]).into_owned_row()); + let is_finished = row[pk_len].clone().map_or(false, |d| d.into_bool()); + let row_count = row + .get(pk_len + 1) + .cloned() + .unwrap_or(None) + .map_or(0, |d| d.into_int64() as u64); + BackfillState { + current_pos, + is_finished, + row_count, + old_state: Some(old_state), + } + } + /// Snapshot read the upstream mv. /// The rows from upstream snapshot read will be buffered inside the `builder`. /// If snapshot is dropped before its rows are consumed, diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 8a2cded5ca8d3..259b67d5f202b 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -308,79 +308,6 @@ pub(crate) async fn get_progress_per_vnode Option { - let datum = if row.len() == state_len - 2 { - // Handle backwards compatibility case where - // we did not have row count (-1 for this). - // -1 to exclude `vnode` as well. - row.last() - } else { - row.datum_at(row.len() - 2) - }; - datum.map(|d| d.into_bool()) -} - -/// The row here does not include `vnode`, -/// it should have been excluded by setting `value_indices`. -/// Row schema: | `pk_indices` ... | `backfill_finished` | `row_count` -pub(crate) fn get_row_count(row: OwnedRow, state_len: usize) -> u64 { - if row.len() == state_len - 2 { - // Handle backwards compatibility case where - // we did not have row count (-1 for this). - // -1 to exclude `vnode` as well. - return 0; - } - match row.last() { - None => 0, - Some(d) => d.into_int64() as u64, - } -} - -pub(crate) async fn get_row_count_state( - state_table: &StateTableInner, - state_len: usize, -) -> StreamExecutorResult { - let mut vnodes = state_table.vnodes().iter_vnodes_scalar(); - let vnode = vnodes.next().unwrap(); - let key: &[Datum] = &[Some(vnode.into())]; - let row = state_table.get_row(key).await?; - let row_count = match row { - None => 0, - Some(row) => get_row_count(row, state_len), - }; - Ok(row_count) -} - -/// All vnodes should be persisted with status finished. -pub(crate) async fn check_all_vnode_finished( - state_table: &StateTableInner, - state_len: usize, -) -> StreamExecutorResult { - debug_assert!(!state_table.vnode_bitmap().is_empty()); - let vnodes = state_table.vnodes().iter_vnodes_scalar(); - let mut is_finished = true; - for vnode in vnodes { - let key: &[Datum] = &[Some(vnode.into())]; - let row = state_table.get_row(key).await?; - - let vnode_is_finished = if let Some(row) = row - && let Some(vnode_is_finished) = get_backfill_finished(row, state_len) - { - vnode_is_finished - } else { - false - }; - if !vnode_is_finished { - is_finished = false; - break; - } - } - Ok(is_finished) -} - /// Flush the data // This is a clippy bug, see https://github.com/rust-lang/rust-clippy/issues/11380. // TODO: remove `allow` here after the issued is closed. diff --git a/src/stream/src/executor/chain.rs b/src/stream/src/executor/chain.rs index ab3ef9ae44973..a51c9e95abbb1 100644 --- a/src/stream/src/executor/chain.rs +++ b/src/stream/src/executor/chain.rs @@ -79,7 +79,7 @@ impl ChainExecutor { // If the barrier is a conf change of creating this mview, and the snapshot is not to be // consumed, we can finish the progress immediately. if barrier.is_newly_added(self.actor_id) && self.upstream_only { - self.progress.finish(barrier.epoch.curr); + self.progress.finish(barrier.epoch.curr, 0); } // The first barrier message should be propagated. @@ -103,7 +103,7 @@ impl ChainExecutor { for msg in upstream { let msg = msg?; if to_consume_snapshot && let Message::Barrier(barrier) = &msg { - self.progress.finish(barrier.epoch.curr); + self.progress.finish(barrier.epoch.curr, 0); } yield msg; } diff --git a/src/stream/src/executor/rearranged_chain.rs b/src/stream/src/executor/rearranged_chain.rs index 1ad43de432551..d2aaae9fd5025 100644 --- a/src/stream/src/executor/rearranged_chain.rs +++ b/src/stream/src/executor/rearranged_chain.rs @@ -135,6 +135,8 @@ impl RearrangedChainExecutor { .unbounded_send(RearrangedMessage::PhantomBarrier(first_barrier)) .unwrap(); + let mut processed_rows: u64 = 0; + { // 3. Rearrange stream, will yield the barriers polled from upstream to rearrange. let rearranged_barrier = @@ -162,8 +164,6 @@ impl RearrangedChainExecutor { let mut last_rearranged_epoch = create_epoch; let mut stop_rearrange_tx = Some(stop_rearrange_tx); - let mut processed_rows: u64 = 0; - #[for_await] for rearranged_msg in &mut rearranged { match rearranged_msg? { @@ -223,7 +223,7 @@ impl RearrangedChainExecutor { continue; }; if let Some(barrier) = msg.as_barrier() { - self.progress.finish(barrier.epoch.curr); + self.progress.finish(barrier.epoch.curr, processed_rows); } yield msg; } @@ -236,7 +236,7 @@ impl RearrangedChainExecutor { for msg in upstream { let msg: Message = msg?; if let Some(barrier) = msg.as_barrier() { - self.progress.finish(barrier.epoch.curr); + self.progress.finish(barrier.epoch.curr, processed_rows); } yield msg; } diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 512e9f6c28da3..1274acee3dac7 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -123,7 +123,7 @@ impl ValuesExecutor { while let Some(barrier) = barrier_receiver.recv().await { if emit { - progress.finish(barrier.epoch.curr); + progress.finish(barrier.epoch.curr, 0); } yield Message::Barrier(barrier); } diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index c438272033831..2c14d6672eb69 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -112,14 +112,14 @@ impl ManagedBarrierState { .into_iter() .map(|(actor, state)| CreateMviewProgress { chain_actor_id: actor, - done: matches!(state, ChainState::Done), + done: matches!(state, ChainState::Done(_)), consumed_epoch: match state { ChainState::ConsumingUpstream(consumed_epoch, _) => consumed_epoch, - ChainState::Done => epoch, + ChainState::Done(_) => epoch, }, consumed_rows: match state { ChainState::ConsumingUpstream(_, consumed_rows) => consumed_rows, - ChainState::Done => 0, + ChainState::Done(consumed_rows) => consumed_rows, }, }) .collect(); diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index adea59cdf656a..5abeab216cd00 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -23,7 +23,7 @@ type ConsumedRows = u64; #[derive(Debug, Clone, Copy)] pub(super) enum ChainState { ConsumingUpstream(ConsumedEpoch, ConsumedRows), - Done, + Done(ConsumedRows), } impl LocalBarrierManager { @@ -129,10 +129,15 @@ impl CreateMviewProgress { ) { match self.state { Some(ChainState::ConsumingUpstream(last, last_consumed_rows)) => { - assert!(last < consumed_epoch); + assert!( + last < consumed_epoch, + "last_epoch: {:#?} must be greater than consumed epoch: {:#?}", + last, + consumed_epoch + ); assert!(last_consumed_rows <= current_consumed_rows); } - Some(ChainState::Done) => unreachable!(), + Some(ChainState::Done(_)) => unreachable!(), None => {} }; self.update_inner( @@ -143,11 +148,11 @@ impl CreateMviewProgress { /// Finish the progress. If the progress is already finished, then perform no-op. /// `current_epoch` should be provided to locate the barrier under concurrent checkpoint. - pub fn finish(&mut self, current_epoch: u64) { - if let Some(ChainState::Done) = self.state { + pub fn finish(&mut self, current_epoch: u64, current_consumed_rows: ConsumedRows) { + if let Some(ChainState::Done(_)) = self.state { return; } - self.update_inner(current_epoch, ChainState::Done); + self.update_inner(current_epoch, ChainState::Done(current_consumed_rows)); } } From 417561d3989374bf795c83687d38203fd717f113 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Sep 2023 14:12:47 +0800 Subject: [PATCH 12/17] feat(test): support customized source logic in deterministic test (#12456) --- Cargo.lock | 1 + src/connector/src/macros.rs | 5 +- src/connector/src/source/mod.rs | 2 + src/connector/src/source/test_source.rs | 239 ++++++++++++++++++ src/frontend/src/handler/create_source.rs | 4 + .../src/executor/wrapper/schema_check.rs | 2 +- src/tests/simulation/Cargo.toml | 1 + .../tests/integration_tests/sink/basic.rs | 114 +++++++-- 8 files changed, 345 insertions(+), 23 deletions(-) create mode 100644 src/connector/src/source/test_source.rs diff --git a/Cargo.lock b/Cargo.lock index 42a08e60f43d5..919c5a234dc00 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7473,6 +7473,7 @@ dependencies = [ "tempfile", "tikv-jemallocator", "tokio-postgres", + "tokio-stream", "tracing", "tracing-subscriber", ] diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 792e2066abcca..62a3cfdcd9682 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -31,7 +31,8 @@ macro_rules! for_all_classified_sources { { Datagen, $crate::source::datagen::DatagenProperties, $crate::source::datagen::DatagenSplit }, { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, - { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit } + { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, + { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( ,$extra_args @@ -152,7 +153,7 @@ macro_rules! dispatch_split_impl { macro_rules! impl_split { ({$({ $variant_name:ident, $prop_name:ty, $split:ty}),*}) => { - #[derive(Debug, Clone, EnumAsInner, PartialEq, Hash)] + #[derive(Debug, Clone, EnumAsInner, PartialEq)] pub enum SplitImpl { $( $variant_name($split), diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 20a9f706e60b5..762af05cd0c96 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -34,6 +34,8 @@ mod common; pub mod external; mod manager; mod mock_external_table; +pub mod test_source; + pub use manager::{SourceColumnDesc, SourceColumnType}; pub use mock_external_table::MockExternalTableReader; diff --git a/src/connector/src/source/test_source.rs b/src/connector/src/source/test_source.rs new file mode 100644 index 0000000000000..743ae3b179427 --- /dev/null +++ b/src/connector/src/source/test_source.rs @@ -0,0 +1,239 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::{Arc, OnceLock}; + +use anyhow::anyhow; +use async_trait::async_trait; +use parking_lot::Mutex; +use risingwave_common::types::JsonbVal; +use serde_derive::{Deserialize, Serialize}; + +use crate::parser::ParserConfig; +use crate::source::{ + BoxSourceWithStateStream, Column, SourceContextRef, SourceEnumeratorContextRef, + SourceProperties, SplitEnumerator, SplitId, SplitMetaData, SplitReader, TryFromHashmap, +}; + +pub type BoxListSplits = Box< + dyn FnMut( + TestSourceProperties, + SourceEnumeratorContextRef, + ) -> anyhow::Result> + + Send + + 'static, +>; + +pub type BoxIntoSourceStream = Box< + dyn FnMut( + TestSourceProperties, + Vec, + ParserConfig, + SourceContextRef, + Option>, + ) -> BoxSourceWithStateStream + + Send + + 'static, +>; + +pub struct BoxSource { + list_split: BoxListSplits, + into_source_stream: BoxIntoSourceStream, +} + +impl BoxSource { + pub fn new( + list_splits: impl FnMut( + TestSourceProperties, + SourceEnumeratorContextRef, + ) -> anyhow::Result> + + Send + + 'static, + into_source_stream: impl FnMut( + TestSourceProperties, + Vec, + ParserConfig, + SourceContextRef, + Option>, + ) -> BoxSourceWithStateStream + + Send + + 'static, + ) -> BoxSource { + BoxSource { + list_split: Box::new(list_splits), + into_source_stream: Box::new(into_source_stream), + } + } +} + +struct TestSourceRegistry { + box_source: Arc>>, +} + +impl TestSourceRegistry { + fn new() -> Self { + TestSourceRegistry { + box_source: Arc::new(Mutex::new(None)), + } + } +} + +fn get_registry() -> &'static TestSourceRegistry { + static GLOBAL_REGISTRY: OnceLock = OnceLock::new(); + GLOBAL_REGISTRY.get_or_init(TestSourceRegistry::new) +} + +pub struct TestSourceRegistryGuard; + +impl Drop for TestSourceRegistryGuard { + fn drop(&mut self) { + assert!(get_registry().box_source.lock().take().is_some()); + } +} + +pub fn registry_test_source(box_source: BoxSource) -> TestSourceRegistryGuard { + assert!(get_registry() + .box_source + .lock() + .replace(box_source) + .is_none()); + TestSourceRegistryGuard +} + +pub const TEST_CONNECTOR: &str = "test"; + +#[derive(Clone, Debug)] +pub struct TestSourceProperties { + properties: HashMap, +} + +impl TryFromHashmap for TestSourceProperties { + fn try_from_hashmap(props: HashMap) -> anyhow::Result { + if cfg!(any(madsim, test)) { + Ok(TestSourceProperties { properties: props }) + } else { + Err(anyhow!("test source only available at test")) + } + } +} + +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +pub struct TestSourceSplit { + pub id: SplitId, + pub properties: HashMap, + pub offset: String, +} + +impl SplitMetaData for TestSourceSplit { + fn id(&self) -> SplitId { + self.id.clone() + } + + fn encode_to_json(&self) -> JsonbVal { + serde_json::to_value(self.clone()).unwrap().into() + } + + fn restore_from_json(value: JsonbVal) -> anyhow::Result { + serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + self.offset = start_offset; + Ok(()) + } +} + +pub struct TestSourceSplitEnumerator { + properties: TestSourceProperties, + context: SourceEnumeratorContextRef, +} + +#[async_trait] +impl SplitEnumerator for TestSourceSplitEnumerator { + type Properties = TestSourceProperties; + type Split = TestSourceSplit; + + async fn new( + properties: Self::Properties, + context: SourceEnumeratorContextRef, + ) -> anyhow::Result { + Ok(Self { + properties, + context, + }) + } + + async fn list_splits(&mut self) -> anyhow::Result> { + (get_registry() + .box_source + .lock() + .as_mut() + .expect("should have init") + .list_split)(self.properties.clone(), self.context.clone()) + } +} + +pub struct TestSourceSplitReader { + properties: TestSourceProperties, + state: Vec, + parser_config: ParserConfig, + source_ctx: SourceContextRef, + columns: Option>, +} + +#[async_trait] +impl SplitReader for TestSourceSplitReader { + type Properties = TestSourceProperties; + type Split = TestSourceSplit; + + async fn new( + properties: Self::Properties, + state: Vec, + parser_config: ParserConfig, + source_ctx: SourceContextRef, + columns: Option>, + ) -> anyhow::Result { + Ok(Self { + properties, + state, + parser_config, + source_ctx, + columns, + }) + } + + fn into_stream(self) -> BoxSourceWithStateStream { + (get_registry() + .box_source + .lock() + .as_mut() + .expect("should have init") + .into_source_stream)( + self.properties, + self.state, + self.parser_config, + self.source_ctx, + self.columns, + ) + } +} + +impl SourceProperties for TestSourceProperties { + type Split = TestSourceSplit; + type SplitEnumerator = TestSourceSplitEnumerator; + type SplitReader = TestSourceSplitReader; + + const SOURCE_NAME: &'static str = TEST_CONNECTOR; +} diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 448d2f49923f7..7479348c4b80f 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -36,6 +36,7 @@ use risingwave_connector::source::cdc::{ use risingwave_connector::source::datagen::DATAGEN_CONNECTOR; use risingwave_connector::source::filesystem::S3_CONNECTOR; use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType}; +use risingwave_connector::source::test_source::TEST_CONNECTOR; use risingwave_connector::source::{ SourceEncode, SourceFormat, SourceStruct, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, PULSAR_CONNECTOR, @@ -907,6 +908,9 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock hashmap!( Format::Plain => vec![Encode::Json], ), + TEST_CONNECTOR => hashmap!( + Format::Plain => vec![Encode::Json], + ) )) }); diff --git a/src/stream/src/executor/wrapper/schema_check.rs b/src/stream/src/executor/wrapper/schema_check.rs index d23eca2b455c6..3e8738db8327a 100644 --- a/src/stream/src/executor/wrapper/schema_check.rs +++ b/src/stream/src/executor/wrapper/schema_check.rs @@ -45,7 +45,7 @@ pub async fn schema_check(info: Arc, input: impl MessageStream) { } Message::Barrier(_) => Ok(()), } - .unwrap_or_else(|e| panic!("schema check failed on {}: {}", info.identity, e)); + .unwrap_or_else(|e| panic!("schema check failed on {:?}: {}", info, e)); yield message; } diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index 26fce12ce37b4..1268b670471d5 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -48,6 +48,7 @@ sqllogictest = "0.17.0" tempfile = "3" tokio = { version = "0.2.23", package = "madsim-tokio" } tokio-postgres = "0.7" +tokio-stream = "0.1" tracing = "0.1" tracing-subscriber = { version = "0.3", features = ["env-filter"] } diff --git a/src/tests/simulation/tests/integration_tests/sink/basic.rs b/src/tests/simulation/tests/integration_tests/sink/basic.rs index c0f9f7253f373..a5715a8471c44 100644 --- a/src/tests/simulation/tests/integration_tests/sink/basic.rs +++ b/src/tests/simulation/tests/integration_tests/sink/basic.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::io::Write; +use std::iter::once; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::Relaxed; use std::sync::Arc; @@ -20,15 +21,23 @@ use std::time::Duration; use anyhow::Result; use async_trait::async_trait; +use futures::stream::select_all; +use futures::StreamExt; use itertools::Itertools; use rand::prelude::SliceRandom; -use risingwave_common::array::StreamChunk; +use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_connector::sink::boxed::{BoxCoordinator, BoxWriter}; use risingwave_connector::sink::test_sink::registry_build_sink; use risingwave_connector::sink::{Sink, SinkWriter, SinkWriterParam}; +use risingwave_connector::source::test_source::{registry_test_source, BoxSource, TestSourceSplit}; +use risingwave_connector::source::StreamChunkWithState; use risingwave_simulation::cluster::{Cluster, ConfigPath, Configuration}; +use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver}; use tokio::time::sleep; +use tokio_stream::wrappers::UnboundedReceiverStream; struct TestWriter { row_counter: Arc, @@ -91,6 +100,21 @@ impl Sink for TestSink { } } +fn build_stream_chunk(row_iter: impl Iterator) -> StreamChunk { + let mut builder = DataChunkBuilder::new(vec![DataType::Int32, DataType::Varchar], 100000); + for (id, name) in row_iter { + assert!(builder + .append_one_row([ + Some(ScalarImpl::Int32(id)), + Some(ScalarImpl::Utf8(name.into())), + ]) + .is_none()); + } + let chunk = builder.consume_all().unwrap(); + let ops = (0..chunk.cardinality()).map(|_| Op::Insert).collect_vec(); + StreamChunk::from_parts(ops, chunk) +} + #[tokio::test] async fn test_sink_basic() -> Result<()> { let config_path = { @@ -126,30 +150,55 @@ async fn test_sink_basic() -> Result<()> { } }); + let source_parallelism = 12; + let mut txs = Vec::new(); + let mut rxs = Vec::new(); + for _ in 0..source_parallelism { + let (tx, rx): (_, UnboundedReceiver) = unbounded_channel(); + txs.push(tx); + rxs.push(Some(rx)); + } + + let _source_guard = registry_test_source(BoxSource::new( + move |_, _| { + Ok((0..source_parallelism) + .map(|i: usize| TestSourceSplit { + id: format!("{}", i).as_str().into(), + properties: Default::default(), + offset: "".to_string(), + }) + .collect_vec()) + }, + move |_, splits, _, _, _| { + select_all(splits.into_iter().map(|split| { + let id: usize = split.id.parse().unwrap(); + let rx = rxs[id].take().unwrap(); + UnboundedReceiverStream::new(rx).map(|chunk| Ok(StreamChunkWithState::from(chunk))) + })) + .boxed() + }, + )); + let mut session = cluster.start_session(); session.run("set streaming_parallelism = 6").await?; session.run("set sink_decouple = false").await?; session - .run("create table test_table (id int, name varchar)") + .run("create table test_table (id int primary key, name varchar) with (connector = 'test') FORMAT PLAIN ENCODE JSON") .await?; session .run("create sink test_sink from test_table with (connector = 'test')") .await?; let mut count = 0; - let mut id_list = (0..100000).collect_vec(); + let mut id_list: Vec = (0..100000).collect_vec(); id_list.shuffle(&mut rand::thread_rng()); let flush_freq = 50; - for id in &id_list[0..1000] { - session - .run(format!( - "insert into test_table values ({}, 'name-{}')", - id, id - )) - .await?; + for id in &id_list[0..10000] { + let chunk = build_stream_chunk(once((*id as i32, format!("name-{}", id)))); + txs[id % source_parallelism].send(chunk).unwrap(); count += 1; if count % flush_freq == 0 { - session.run("flush").await?; + sleep(Duration::from_millis(10)).await; } } @@ -198,12 +247,41 @@ async fn test_sink_decouple_basic() -> Result<()> { } }); + let source_parallelism = 12; + let mut txs = Vec::new(); + let mut rxs = Vec::new(); + for _ in 0..source_parallelism { + let (tx, rx): (_, UnboundedReceiver) = unbounded_channel(); + txs.push(tx); + rxs.push(Some(rx)); + } + + let _source_guard = registry_test_source(BoxSource::new( + move |_, _| { + Ok((0..source_parallelism) + .map(|i: usize| TestSourceSplit { + id: format!("{}", i).as_str().into(), + properties: Default::default(), + offset: "".to_string(), + }) + .collect_vec()) + }, + move |_, splits, _, _, _| { + select_all(splits.into_iter().map(|split| { + let id: usize = split.id.parse().unwrap(); + let rx = rxs[id].take().unwrap(); + UnboundedReceiverStream::new(rx).map(|chunk| Ok(StreamChunkWithState::from(chunk))) + })) + .boxed() + }, + )); + let mut session = cluster.start_session(); session.run("set streaming_parallelism = 6").await?; session.run("set sink_decouple = true").await?; session - .run("create table test_table (id int, name varchar)") + .run("create table test_table (id int primary key, name varchar) with (connector = 'test') FORMAT PLAIN ENCODE JSON") .await?; session .run("create sink test_sink from test_table with (connector = 'test')") @@ -214,16 +292,12 @@ async fn test_sink_decouple_basic() -> Result<()> { let mut id_list = (0..100000).collect_vec(); id_list.shuffle(&mut rand::thread_rng()); let flush_freq = 50; - for id in &id_list[0..1000] { - session - .run(format!( - "insert into test_table values ({}, 'name-{}')", - id, id - )) - .await?; + for id in &id_list[0..10000] { + let chunk = build_stream_chunk(once((*id as i32, format!("name-{}", id)))); + txs[id % source_parallelism].send(chunk).unwrap(); count += 1; if count % flush_freq == 0 { - session.run("flush").await?; + sleep(Duration::from_millis(10)).await; } } From d979e88b9575a8878c59779b094b6d51deba4b43 Mon Sep 17 00:00:00 2001 From: Li0k Date: Tue, 26 Sep 2023 14:22:03 +0800 Subject: [PATCH 13/17] refactor(storage): reorg selector (#12392) --- .../src/hummock/compaction/level_selector.rs | 448 +---------- src/meta/src/hummock/compaction/mod.rs | 89 +-- .../picker/base_level_compaction_picker.rs | 7 +- .../picker/intra_compaction_picker.rs | 2 +- .../picker/manual_compaction_picker.rs | 8 +- .../picker/min_overlap_compaction_picker.rs | 4 +- .../picker/space_reclaim_compaction_picker.rs | 7 +- .../picker/tier_compaction_picker.rs | 6 +- .../tombstone_reclaim_compaction_picker.rs | 2 +- .../picker/ttl_reclaim_compaction_picker.rs | 4 +- .../compaction/selector/emergency_selector.rs | 70 ++ .../compaction/selector/level_selector.rs | 710 ++++++++++++++++++ .../compaction/selector/manual_selector.rs | 122 +++ .../src/hummock/compaction/selector/mod.rs | 348 +++++++++ .../selector/space_reclaim_selector.rs | 74 ++ .../tombstone_compaction_selector.rs | 6 +- .../compaction/selector/ttl_selector.rs | 73 ++ src/meta/src/hummock/compactor_manager.rs | 4 +- src/meta/src/hummock/manager/mod.rs | 25 +- src/meta/src/hummock/manager/tests.rs | 36 +- src/meta/src/hummock/metrics_utils.rs | 3 +- .../src/hummock/mock_hummock_meta_client.rs | 10 +- src/meta/src/hummock/test_utils.rs | 4 +- src/meta/src/rpc/service/hummock_service.rs | 2 +- .../hummock_test/src/compactor_tests.rs | 16 +- .../hummock_test/src/sync_point_tests.rs | 2 +- 26 files changed, 1483 insertions(+), 599 deletions(-) create mode 100644 src/meta/src/hummock/compaction/selector/emergency_selector.rs create mode 100644 src/meta/src/hummock/compaction/selector/level_selector.rs create mode 100644 src/meta/src/hummock/compaction/selector/manual_selector.rs create mode 100644 src/meta/src/hummock/compaction/selector/mod.rs create mode 100644 src/meta/src/hummock/compaction/selector/space_reclaim_selector.rs rename src/meta/src/hummock/compaction/{ => selector}/tombstone_compaction_selector.rs (94%) create mode 100644 src/meta/src/hummock/compaction/selector/ttl_selector.rs diff --git a/src/meta/src/hummock/compaction/level_selector.rs b/src/meta/src/hummock/compaction/level_selector.rs index 05975f64da1b1..5f673aac0cd2e 100644 --- a/src/meta/src/hummock/compaction/level_selector.rs +++ b/src/meta/src/hummock/compaction/level_selector.rs @@ -31,8 +31,8 @@ use super::picker::{ TtlReclaimCompactionPicker, }; use super::{ - create_compaction_task, LevelCompactionPicker, ManualCompactionOption, ManualCompactionPicker, - TierCompactionPicker, + create_compaction_task, CompactionSelector, LevelCompactionPicker, ManualCompactionOption, + ManualCompactionPicker, TierCompactionPicker, }; use crate::hummock::compaction::overlap_strategy::OverlapStrategy; use crate::hummock::compaction::picker::{ @@ -41,7 +41,6 @@ use crate::hummock::compaction::picker::{ use crate::hummock::compaction::{create_overlap_strategy, CompactionTask, LocalSelectorStatistic}; use crate::hummock::level_handler::LevelHandler; use crate::hummock::model::CompactionGroup; -use crate::rpc::metrics::MetaMetrics; pub const SCORE_BASE: u64 = 100; @@ -62,24 +61,6 @@ pub struct PickerInfo { picker_type: PickerType, } -pub trait LevelSelector: Sync + Send { - fn pick_compaction( - &mut self, - task_id: HummockCompactionTaskId, - group: &CompactionGroup, - levels: &Levels, - level_handlers: &mut [LevelHandler], - selector_stats: &mut LocalSelectorStatistic, - table_id_to_options: HashMap, - ) -> Option; - - fn report_statistic_metrics(&self, _metrics: &MetaMetrics) {} - - fn name(&self) -> &'static str; - - fn task_type(&self) -> compact_task::TaskType; -} - #[derive(Default, Debug)] pub struct SelectContext { pub level_max_bytes: Vec, @@ -410,7 +391,7 @@ impl DynamicLevelSelectorCore { } } -impl LevelSelector for DynamicLevelSelector { +impl CompactionSelector for DynamicLevelSelector { fn pick_compaction( &mut self, task_id: HummockCompactionTaskId, @@ -467,200 +448,6 @@ impl LevelSelector for DynamicLevelSelector { } } -pub struct ManualCompactionSelector { - option: ManualCompactionOption, -} - -impl ManualCompactionSelector { - pub fn new(option: ManualCompactionOption) -> Self { - Self { option } - } -} - -impl LevelSelector for ManualCompactionSelector { - fn pick_compaction( - &mut self, - task_id: HummockCompactionTaskId, - group: &CompactionGroup, - levels: &Levels, - level_handlers: &mut [LevelHandler], - _selector_stats: &mut LocalSelectorStatistic, - _table_id_to_options: HashMap, - ) -> Option { - let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); - let overlap_strategy = create_overlap_strategy(group.compaction_config.compaction_mode()); - let ctx = dynamic_level_core.calculate_level_base_size(levels); - let (mut picker, base_level) = { - let target_level = if self.option.level == 0 { - ctx.base_level - } else if self.option.level == group.compaction_config.max_level as usize { - self.option.level - } else { - self.option.level + 1 - }; - if self.option.level > 0 && self.option.level < ctx.base_level { - return None; - } - ( - ManualCompactionPicker::new(overlap_strategy, self.option.clone(), target_level), - ctx.base_level, - ) - }; - - let compaction_input = - picker.pick_compaction(levels, level_handlers, &mut LocalPickerStatistic::default())?; - compaction_input.add_pending_task(task_id, level_handlers); - - Some(create_compaction_task( - group.compaction_config.as_ref(), - compaction_input, - base_level, - self.task_type(), - )) - } - - fn name(&self) -> &'static str { - "ManualCompactionSelector" - } - - fn task_type(&self) -> compact_task::TaskType { - compact_task::TaskType::Manual - } -} - -#[derive(Default)] -pub struct SpaceReclaimCompactionSelector { - state: HashMap, -} - -impl LevelSelector for SpaceReclaimCompactionSelector { - fn pick_compaction( - &mut self, - task_id: HummockCompactionTaskId, - group: &CompactionGroup, - levels: &Levels, - level_handlers: &mut [LevelHandler], - _selector_stats: &mut LocalSelectorStatistic, - _table_id_to_options: HashMap, - ) -> Option { - let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); - let mut picker = SpaceReclaimCompactionPicker::new( - group.compaction_config.max_space_reclaim_bytes, - levels.member_table_ids.iter().cloned().collect(), - ); - let ctx = dynamic_level_core.calculate_level_base_size(levels); - let state = self.state.entry(group.group_id).or_default(); - - let compaction_input = picker.pick_compaction(levels, level_handlers, state)?; - compaction_input.add_pending_task(task_id, level_handlers); - - Some(create_compaction_task( - dynamic_level_core.get_config(), - compaction_input, - ctx.base_level, - self.task_type(), - )) - } - - fn name(&self) -> &'static str { - "SpaceReclaimCompaction" - } - - fn task_type(&self) -> compact_task::TaskType { - compact_task::TaskType::SpaceReclaim - } -} - -#[derive(Default)] -pub struct TtlCompactionSelector { - state: HashMap, -} - -impl LevelSelector for TtlCompactionSelector { - fn pick_compaction( - &mut self, - task_id: HummockCompactionTaskId, - group: &CompactionGroup, - levels: &Levels, - level_handlers: &mut [LevelHandler], - _selector_stats: &mut LocalSelectorStatistic, - table_id_to_options: HashMap, - ) -> Option { - let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); - let ctx = dynamic_level_core.calculate_level_base_size(levels); - let picker = TtlReclaimCompactionPicker::new( - group.compaction_config.max_space_reclaim_bytes, - table_id_to_options, - ); - let state = self.state.entry(group.group_id).or_default(); - let compaction_input = picker.pick_compaction(levels, level_handlers, state)?; - compaction_input.add_pending_task(task_id, level_handlers); - - Some(create_compaction_task( - group.compaction_config.as_ref(), - compaction_input, - ctx.base_level, - self.task_type(), - )) - } - - fn name(&self) -> &'static str { - "TtlCompaction" - } - - fn task_type(&self) -> compact_task::TaskType { - compact_task::TaskType::Ttl - } -} - -pub fn default_level_selector() -> Box { - Box::::default() -} - -#[derive(Default)] -pub struct EmergencySelector {} - -impl LevelSelector for EmergencySelector { - fn pick_compaction( - &mut self, - task_id: HummockCompactionTaskId, - group: &CompactionGroup, - levels: &Levels, - level_handlers: &mut [LevelHandler], - selector_stats: &mut LocalSelectorStatistic, - _table_id_to_options: HashMap, - ) -> Option { - let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); - let ctx = dynamic_level_core.calculate_level_base_size(levels); - let picker = - EmergencyCompactionPicker::new(ctx.base_level, group.compaction_config.clone()); - - let mut stats = LocalPickerStatistic::default(); - if let Some(compaction_input) = picker.pick_compaction(levels, level_handlers, &mut stats) { - compaction_input.add_pending_task(task_id, level_handlers); - - return Some(create_compaction_task( - group.compaction_config.as_ref(), - compaction_input, - ctx.base_level, - self.task_type(), - )); - } - - selector_stats.skip_picker.push((0, ctx.base_level, stats)); - - None - } - - fn name(&self) -> &'static str { - "EmergencyCompaction" - } - - fn task_type(&self) -> compact_task::TaskType { - compact_task::TaskType::Emergency - } -} - #[cfg(test)] pub mod tests { use std::ops::Range; @@ -674,235 +461,6 @@ pub mod tests { use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; use crate::hummock::test_utils::iterator_test_key_of_epoch; - pub fn push_table_level0_overlapping(levels: &mut Levels, sst: SstableInfo) { - levels.l0.as_mut().unwrap().total_file_size += sst.file_size; - levels.l0.as_mut().unwrap().sub_levels.push(Level { - level_idx: 0, - level_type: LevelType::Overlapping as i32, - total_file_size: sst.file_size, - uncompressed_file_size: sst.uncompressed_file_size, - sub_level_id: sst.get_sst_id(), - table_infos: vec![sst], - }); - } - - pub fn push_table_level0_nonoverlapping(levels: &mut Levels, sst: SstableInfo) { - push_table_level0_overlapping(levels, sst); - levels - .l0 - .as_mut() - .unwrap() - .sub_levels - .last_mut() - .unwrap() - .level_type = LevelType::Nonoverlapping as i32; - } - - pub fn push_tables_level0_nonoverlapping(levels: &mut Levels, table_infos: Vec) { - let total_file_size = table_infos.iter().map(|table| table.file_size).sum::(); - let uncompressed_file_size = table_infos - .iter() - .map(|table| table.uncompressed_file_size) - .sum(); - let sub_level_id = table_infos[0].get_sst_id(); - levels.l0.as_mut().unwrap().total_file_size += total_file_size; - levels.l0.as_mut().unwrap().sub_levels.push(Level { - level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, - total_file_size, - sub_level_id, - table_infos, - uncompressed_file_size, - }); - } - - pub fn generate_table( - id: u64, - table_prefix: u64, - left: usize, - right: usize, - epoch: u64, - ) -> SstableInfo { - SstableInfo { - object_id: id, - sst_id: id, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(table_prefix, left, epoch), - right: iterator_test_key_of_epoch(table_prefix, right, epoch), - right_exclusive: false, - }), - file_size: (right - left + 1) as u64, - table_ids: vec![table_prefix as u32], - uncompressed_file_size: (right - left + 1) as u64, - total_key_count: (right - left + 1) as u64, - ..Default::default() - } - } - - #[allow(clippy::too_many_arguments)] - pub fn generate_table_with_ids_and_epochs( - id: u64, - table_prefix: u64, - left: usize, - right: usize, - epoch: u64, - table_ids: Vec, - min_epoch: u64, - max_epoch: u64, - ) -> SstableInfo { - SstableInfo { - object_id: id, - sst_id: id, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(table_prefix, left, epoch), - right: iterator_test_key_of_epoch(table_prefix, right, epoch), - right_exclusive: false, - }), - file_size: (right - left + 1) as u64, - table_ids, - uncompressed_file_size: (right - left + 1) as u64, - min_epoch, - max_epoch, - ..Default::default() - } - } - - pub fn generate_tables( - ids: Range, - keys: Range, - epoch: u64, - file_size: u64, - ) -> Vec { - let step = (keys.end - keys.start) / (ids.end - ids.start) as usize; - let mut start = keys.start; - let mut tables = vec![]; - for id in ids { - let mut table = generate_table(id, 1, start, start + step - 1, epoch); - table.file_size = file_size; - tables.push(table); - start += step; - } - tables - } - - pub fn generate_level(level_idx: u32, table_infos: Vec) -> Level { - let total_file_size = table_infos.iter().map(|sst| sst.file_size).sum(); - let uncompressed_file_size = table_infos - .iter() - .map(|sst| sst.uncompressed_file_size) - .sum(); - Level { - level_idx, - level_type: LevelType::Nonoverlapping as i32, - table_infos, - total_file_size, - sub_level_id: 0, - uncompressed_file_size, - } - } - - /// Returns a `OverlappingLevel`, with each `table_infos`'s element placed in a nonoverlapping - /// sub-level. - pub fn generate_l0_nonoverlapping_sublevels(table_infos: Vec) -> OverlappingLevel { - let total_file_size = table_infos.iter().map(|table| table.file_size).sum::(); - let uncompressed_file_size = table_infos - .iter() - .map(|table| table.uncompressed_file_size) - .sum::(); - OverlappingLevel { - sub_levels: table_infos - .into_iter() - .enumerate() - .map(|(idx, table)| Level { - level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, - total_file_size: table.file_size, - uncompressed_file_size: table.uncompressed_file_size, - sub_level_id: idx as u64, - table_infos: vec![table], - }) - .collect_vec(), - total_file_size, - uncompressed_file_size, - } - } - - pub fn generate_l0_nonoverlapping_multi_sublevels( - table_infos: Vec>, - ) -> OverlappingLevel { - let mut l0 = OverlappingLevel { - sub_levels: table_infos - .into_iter() - .enumerate() - .map(|(idx, table)| Level { - level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, - total_file_size: table.iter().map(|table| table.file_size).sum::(), - uncompressed_file_size: table - .iter() - .map(|sst| sst.uncompressed_file_size) - .sum::(), - sub_level_id: idx as u64, - table_infos: table, - }) - .collect_vec(), - total_file_size: 0, - uncompressed_file_size: 0, - }; - - l0.total_file_size = l0.sub_levels.iter().map(|l| l.total_file_size).sum::(); - l0.uncompressed_file_size = l0 - .sub_levels - .iter() - .map(|l| l.uncompressed_file_size) - .sum::(); - l0 - } - - /// Returns a `OverlappingLevel`, with each `table_infos`'s element placed in a overlapping - /// sub-level. - pub fn generate_l0_overlapping_sublevels( - table_infos: Vec>, - ) -> OverlappingLevel { - let mut l0 = OverlappingLevel { - sub_levels: table_infos - .into_iter() - .enumerate() - .map(|(idx, table)| Level { - level_idx: 0, - level_type: LevelType::Overlapping as i32, - total_file_size: table.iter().map(|table| table.file_size).sum::(), - sub_level_id: idx as u64, - table_infos: table.clone(), - uncompressed_file_size: table - .iter() - .map(|sst| sst.uncompressed_file_size) - .sum::(), - }) - .collect_vec(), - total_file_size: 0, - uncompressed_file_size: 0, - }; - l0.total_file_size = l0.sub_levels.iter().map(|l| l.total_file_size).sum::(); - l0.uncompressed_file_size = l0 - .sub_levels - .iter() - .map(|l| l.uncompressed_file_size) - .sum::(); - l0 - } - - pub(crate) fn assert_compaction_task( - compact_task: &CompactionTask, - level_handlers: &[LevelHandler], - ) { - for i in &compact_task.input.input_levels { - for t in &i.table_infos { - assert!(level_handlers[i.level_idx as usize].is_pending_compact(&t.sst_id)); - } - } - } - #[test] fn test_dynamic_level() { let config = CompactionConfigBuilder::new() diff --git a/src/meta/src/hummock/compaction/mod.rs b/src/meta/src/hummock/compaction/mod.rs index 23585da8999a9..632061b6d7705 100644 --- a/src/meta/src/hummock/compaction/mod.rs +++ b/src/meta/src/hummock/compaction/mod.rs @@ -15,37 +15,32 @@ #![expect(clippy::arc_with_non_send_sync, reason = "FIXME: later")] pub mod compaction_config; -mod level_selector; mod overlap_strategy; -mod tombstone_compaction_selector; use risingwave_common::catalog::TableOption; -use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; use risingwave_pb::hummock::compact_task::{self, TaskStatus}; mod picker; +pub mod selector; + use std::collections::{HashMap, HashSet}; use std::fmt::{Debug, Formatter}; use std::sync::Arc; -use picker::{LevelCompactionPicker, ManualCompactionPicker, TierCompactionPicker}; +use picker::{LevelCompactionPicker, TierCompactionPicker}; use risingwave_hummock_sdk::{ - can_concat, CompactionGroupId, HummockCompactionTaskId, HummockEpoch, HummockSstableId, + can_concat, CompactionGroupId, HummockCompactionTaskId, HummockEpoch, }; use risingwave_pb::hummock::compaction_config::CompactionMode; use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::{CompactTask, CompactionConfig, KeyRange, LevelType}; +pub use selector::CompactionSelector; -pub use crate::hummock::compaction::level_selector::{ - default_level_selector, DynamicLevelSelector, DynamicLevelSelectorCore, EmergencySelector, - LevelSelector, ManualCompactionSelector, SpaceReclaimCompactionSelector, TtlCompactionSelector, -}; +use self::selector::LocalSelectorStatistic; use crate::hummock::compaction::overlap_strategy::{OverlapStrategy, RangeOverlapStrategy}; -use crate::hummock::compaction::picker::{CompactionInput, LocalPickerStatistic}; -pub use crate::hummock::compaction::tombstone_compaction_selector::TombstoneCompactionSelector; +use crate::hummock::compaction::picker::CompactionInput; use crate::hummock::level_handler::LevelHandler; use crate::hummock::model::CompactionGroup; -use crate::rpc::metrics::MetaMetrics; pub struct CompactStatus { pub(crate) compaction_group_id: CompactionGroupId, @@ -111,7 +106,7 @@ impl CompactStatus { task_id: HummockCompactionTaskId, group: &CompactionGroup, stats: &mut LocalSelectorStatistic, - selector: &mut Box, + selector: &mut Box, table_id_to_options: HashMap, ) -> Option { // When we compact the files, we must make the result of compaction meet the following @@ -209,74 +204,6 @@ impl CompactStatus { } } -#[derive(Clone, Debug, PartialEq)] -pub struct ManualCompactionOption { - /// Filters out SSTs to pick. Has no effect if empty. - pub sst_ids: Vec, - /// Filters out SSTs to pick. - pub key_range: KeyRange, - /// Filters out SSTs to pick. Has no effect if empty. - pub internal_table_id: HashSet, - /// Input level. - pub level: usize, -} - -impl Default for ManualCompactionOption { - fn default() -> Self { - Self { - sst_ids: vec![], - key_range: KeyRange { - left: vec![], - right: vec![], - right_exclusive: false, - }, - internal_table_id: HashSet::default(), - level: 1, - } - } -} - -#[derive(Default)] -pub struct LocalSelectorStatistic { - skip_picker: Vec<(usize, usize, LocalPickerStatistic)>, -} - -impl LocalSelectorStatistic { - pub fn report_to_metrics(&self, group_id: u64, metrics: &MetaMetrics) { - for (start_level, target_level, stats) in &self.skip_picker { - let level_label = format!("cg{}-{}-to-{}", group_id, start_level, target_level); - if stats.skip_by_write_amp_limit > 0 { - metrics - .compact_skip_frequency - .with_label_values(&[level_label.as_str(), "write-amp"]) - .inc(); - } - if stats.skip_by_count_limit > 0 { - metrics - .compact_skip_frequency - .with_label_values(&[level_label.as_str(), "count"]) - .inc(); - } - if stats.skip_by_pending_files > 0 { - metrics - .compact_skip_frequency - .with_label_values(&[level_label.as_str(), "pending-files"]) - .inc(); - } - if stats.skip_by_overlapping > 0 { - metrics - .compact_skip_frequency - .with_label_values(&[level_label.as_str(), "overlapping"]) - .inc(); - } - metrics - .compact_skip_frequency - .with_label_values(&[level_label.as_str(), "picker"]) - .inc(); - } - } -} - pub fn create_compaction_task( compaction_config: &CompactionConfig, input: CompactionInput, 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 684a6a3125bb8..6e1b33b1935d2 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 @@ -240,12 +240,7 @@ pub mod tests { use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; - use crate::hummock::compaction::level_selector::tests::{ - generate_l0_nonoverlapping_multi_sublevels, generate_l0_nonoverlapping_sublevels, - generate_l0_overlapping_sublevels, generate_level, generate_table, - push_table_level0_nonoverlapping, push_table_level0_overlapping, - push_tables_level0_nonoverlapping, - }; + use crate::hummock::compaction::selector::tests::*; use crate::hummock::compaction::{CompactionMode, TierCompactionPicker}; fn create_compaction_picker_for_test() -> LevelCompactionPicker { 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 541b93254172b..c744c79c89995 100644 --- a/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs @@ -260,7 +260,7 @@ pub mod tests { use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; - use crate::hummock::compaction::level_selector::tests::{ + use crate::hummock::compaction::selector::tests::{ generate_l0_nonoverlapping_multi_sublevels, generate_l0_nonoverlapping_sublevels, generate_l0_overlapping_sublevels, generate_level, generate_table, push_table_level0_overlapping, push_tables_level0_nonoverlapping, diff --git a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs index e8f8c908d0fd3..23b1f0b6a9960 100644 --- a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs @@ -24,7 +24,7 @@ use super::{CompactionInput, CompactionPicker, LocalPickerStatistic}; use crate::hummock::compaction::overlap_strategy::{ OverlapInfo, OverlapStrategy, RangeOverlapInfo, }; -use crate::hummock::compaction::ManualCompactionOption; +use crate::hummock::compaction::selector::ManualCompactionOption; use crate::hummock::level_handler::LevelHandler; pub struct ManualCompactionPicker { @@ -333,12 +333,12 @@ pub mod tests { use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; - use crate::hummock::compaction::level_selector::tests::{ + use crate::hummock::compaction::overlap_strategy::RangeOverlapStrategy; + use crate::hummock::compaction::selector::tests::{ assert_compaction_task, generate_l0_nonoverlapping_sublevels, generate_l0_overlapping_sublevels, generate_level, generate_table, }; - use crate::hummock::compaction::level_selector::{LevelSelector, ManualCompactionSelector}; - use crate::hummock::compaction::overlap_strategy::RangeOverlapStrategy; + use crate::hummock::compaction::selector::{CompactionSelector, ManualCompactionSelector}; use crate::hummock::compaction::LocalSelectorStatistic; use crate::hummock::model::CompactionGroup; use crate::hummock::test_utils::iterator_test_key_of_epoch; 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 0cf44795e0acb..c705bd3b43aae 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 @@ -379,10 +379,10 @@ pub mod tests { pub use risingwave_pb::hummock::{KeyRange, Level, LevelType}; use super::*; - use crate::hummock::compaction::level_selector::tests::{ + use crate::hummock::compaction::overlap_strategy::RangeOverlapStrategy; + use crate::hummock::compaction::selector::tests::{ generate_l0_nonoverlapping_sublevels, generate_table, }; - use crate::hummock::compaction::overlap_strategy::RangeOverlapStrategy; #[test] fn test_compact_l1() { diff --git a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs index a3ff21831fef8..63f3f3536e5e2 100644 --- a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs @@ -174,12 +174,13 @@ mod test { use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; - use crate::hummock::compaction::level_selector::tests::{ + use crate::hummock::compaction::selector::tests::{ assert_compaction_task, generate_l0_nonoverlapping_sublevels, generate_level, generate_table_with_ids_and_epochs, }; - use crate::hummock::compaction::level_selector::SpaceReclaimCompactionSelector; - use crate::hummock::compaction::{LevelSelector, LocalSelectorStatistic}; + use crate::hummock::compaction::selector::{ + CompactionSelector, LocalSelectorStatistic, SpaceReclaimCompactionSelector, + }; use crate::hummock::model::CompactionGroup; #[test] 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 5b3058317a4b0..a64bf489a197a 100644 --- a/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs @@ -163,12 +163,12 @@ pub mod tests { use risingwave_pb::hummock::{LevelType, OverlappingLevel}; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; - use crate::hummock::compaction::level_selector::tests::{ - generate_l0_overlapping_sublevels, generate_table, push_table_level0_overlapping, - }; use crate::hummock::compaction::picker::{ CompactionPicker, LocalPickerStatistic, TierCompactionPicker, }; + use crate::hummock::compaction::selector::tests::{ + generate_l0_overlapping_sublevels, generate_table, push_table_level0_overlapping, + }; use crate::hummock::level_handler::LevelHandler; #[test] diff --git a/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs index 994bfbc5ea557..5438abefa2021 100644 --- a/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs @@ -151,7 +151,7 @@ pub mod tests { use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; use crate::hummock::compaction::create_overlap_strategy; - use crate::hummock::compaction::level_selector::tests::{generate_level, generate_table}; + use crate::hummock::compaction::selector::tests::{generate_level, generate_table}; #[test] fn test_basic() { diff --git a/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs index 9f84b99453f17..07f074a5394b8 100644 --- a/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs @@ -227,11 +227,11 @@ mod test { use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; - use crate::hummock::compaction::level_selector::tests::{ + use crate::hummock::compaction::selector::tests::{ assert_compaction_task, generate_l0_nonoverlapping_sublevels, generate_level, generate_table_with_ids_and_epochs, }; - use crate::hummock::compaction::level_selector::{LevelSelector, TtlCompactionSelector}; + use crate::hummock::compaction::selector::{CompactionSelector, TtlCompactionSelector}; use crate::hummock::compaction::LocalSelectorStatistic; use crate::hummock::model::CompactionGroup; diff --git a/src/meta/src/hummock/compaction/selector/emergency_selector.rs b/src/meta/src/hummock/compaction/selector/emergency_selector.rs new file mode 100644 index 0000000000000..3f5a81e264956 --- /dev/null +++ b/src/meta/src/hummock/compaction/selector/emergency_selector.rs @@ -0,0 +1,70 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; + +use risingwave_common::catalog::TableOption; +use risingwave_hummock_sdk::HummockCompactionTaskId; +use risingwave_pb::hummock::compact_task; +use risingwave_pb::hummock::hummock_version::Levels; + +use super::{CompactionSelector, DynamicLevelSelectorCore, LocalSelectorStatistic}; +use crate::hummock::compaction::picker::{EmergencyCompactionPicker, LocalPickerStatistic}; +use crate::hummock::compaction::{create_compaction_task, CompactionTask}; +use crate::hummock::level_handler::LevelHandler; +use crate::hummock::model::CompactionGroup; + +#[derive(Default)] +pub struct EmergencySelector {} + +impl CompactionSelector for EmergencySelector { + fn pick_compaction( + &mut self, + task_id: HummockCompactionTaskId, + group: &CompactionGroup, + levels: &Levels, + level_handlers: &mut [LevelHandler], + selector_stats: &mut LocalSelectorStatistic, + _table_id_to_options: HashMap, + ) -> Option { + let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); + let ctx = dynamic_level_core.calculate_level_base_size(levels); + let picker = + EmergencyCompactionPicker::new(ctx.base_level, group.compaction_config.clone()); + + let mut stats = LocalPickerStatistic::default(); + if let Some(compaction_input) = picker.pick_compaction(levels, level_handlers, &mut stats) { + compaction_input.add_pending_task(task_id, level_handlers); + + return Some(create_compaction_task( + group.compaction_config.as_ref(), + compaction_input, + ctx.base_level, + self.task_type(), + )); + } + + selector_stats.skip_picker.push((0, ctx.base_level, stats)); + + None + } + + fn name(&self) -> &'static str { + "EmergencyCompaction" + } + + fn task_type(&self) -> compact_task::TaskType { + compact_task::TaskType::Emergency + } +} diff --git a/src/meta/src/hummock/compaction/selector/level_selector.rs b/src/meta/src/hummock/compaction/selector/level_selector.rs new file mode 100644 index 0000000000000..8c65d90ac9c12 --- /dev/null +++ b/src/meta/src/hummock/compaction/selector/level_selector.rs @@ -0,0 +1,710 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +use std::collections::HashMap; +use std::sync::Arc; + +use risingwave_common::catalog::TableOption; +use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockLevelsExt; +use risingwave_hummock_sdk::HummockCompactionTaskId; +use risingwave_pb::hummock::hummock_version::Levels; +use risingwave_pb::hummock::{compact_task, CompactionConfig, LevelType}; + +use super::{ + create_compaction_task, CompactionSelector, LevelCompactionPicker, TierCompactionPicker, +}; +use crate::hummock::compaction::overlap_strategy::OverlapStrategy; +use crate::hummock::compaction::picker::{ + CompactionPicker, CompactionTaskValidator, IntraCompactionPicker, LocalPickerStatistic, + MinOverlappingPicker, +}; +use crate::hummock::compaction::{create_overlap_strategy, CompactionTask, LocalSelectorStatistic}; +use crate::hummock::level_handler::LevelHandler; +use crate::hummock::model::CompactionGroup; + +pub const SCORE_BASE: u64 = 100; + +#[derive(Debug, Default, Clone)] +pub enum PickerType { + Tier, + Intra, + ToBase, + #[default] + BottomLevel, +} + +#[derive(Default, Debug)] +pub struct PickerInfo { + score: u64, + select_level: usize, + target_level: usize, + picker_type: PickerType, +} + +#[derive(Default, Debug)] +pub struct SelectContext { + pub level_max_bytes: Vec, + + // All data will be placed in the last level. When the cluster is empty, the files in L0 will + // be compact to `max_level`, and the `max_level` would be `base_level`. When the total + // size of the files in `base_level` reaches its capacity, we will place data in a higher + // level, which equals to `base_level -= 1;`. + pub base_level: usize, + pub score_levels: Vec, +} + +pub struct DynamicLevelSelectorCore { + config: Arc, +} + +#[derive(Default)] +pub struct DynamicLevelSelector {} + +impl DynamicLevelSelectorCore { + pub fn new(config: Arc) -> Self { + Self { config } + } + + pub fn get_config(&self) -> &CompactionConfig { + self.config.as_ref() + } + + fn create_compaction_picker( + &self, + picker_info: &PickerInfo, + overlap_strategy: Arc, + compaction_task_validator: Arc, + ) -> Box { + match picker_info.picker_type { + PickerType::Tier => Box::new(TierCompactionPicker::new_with_validator( + self.config.clone(), + compaction_task_validator, + )), + PickerType::ToBase => Box::new(LevelCompactionPicker::new_with_validator( + picker_info.target_level, + self.config.clone(), + compaction_task_validator, + )), + PickerType::Intra => Box::new(IntraCompactionPicker::new_with_validator( + self.config.clone(), + compaction_task_validator, + )), + PickerType::BottomLevel => { + assert_eq!(picker_info.select_level + 1, picker_info.target_level); + Box::new(MinOverlappingPicker::new( + picker_info.select_level, + picker_info.target_level, + self.config.max_bytes_for_level_base, + self.config.split_by_state_table, + overlap_strategy, + )) + } + } + } + + // TODO: calculate this scores in apply compact result. + /// `calculate_level_base_size` calculate base level and the base size of LSM tree build for + /// current dataset. In other words, `level_max_bytes` is our compaction goal which shall + /// reach. This algorithm refers to the implementation in [`https://github.com/facebook/rocksdb/blob/v7.2.2/db/version_set.cc#L3706`] + pub fn calculate_level_base_size(&self, levels: &Levels) -> SelectContext { + let mut first_non_empty_level = 0; + let mut max_level_size = 0; + let mut ctx = SelectContext::default(); + + for level in &levels.levels { + if level.total_file_size > 0 && first_non_empty_level == 0 { + first_non_empty_level = level.level_idx as usize; + } + max_level_size = std::cmp::max(max_level_size, level.total_file_size); + } + + ctx.level_max_bytes + .resize(self.config.max_level as usize + 1, u64::MAX); + + if max_level_size == 0 { + // Use the bottommost level. + ctx.base_level = self.config.max_level as usize; + return ctx; + } + + let base_bytes_max = self.config.max_bytes_for_level_base; + let base_bytes_min = base_bytes_max / self.config.max_bytes_for_level_multiplier; + + let mut cur_level_size = max_level_size; + for _ in first_non_empty_level..self.config.max_level as usize { + cur_level_size /= self.config.max_bytes_for_level_multiplier; + } + + let base_level_size = if cur_level_size <= base_bytes_min { + // Case 1. If we make target size of last level to be max_level_size, + // target size of the first non-empty level would be smaller than + // base_bytes_min. We set it be base_bytes_min. + ctx.base_level = first_non_empty_level; + base_bytes_min + 1 + } else { + ctx.base_level = first_non_empty_level; + while ctx.base_level > 1 && cur_level_size > base_bytes_max { + ctx.base_level -= 1; + cur_level_size /= self.config.max_bytes_for_level_multiplier; + } + std::cmp::min(base_bytes_max, cur_level_size) + }; + + let level_multiplier = self.config.max_bytes_for_level_multiplier as f64; + let mut level_size = base_level_size; + for i in ctx.base_level..=self.config.max_level as usize { + // Don't set any level below base_bytes_max. Otherwise, the LSM can + // assume an hourglass shape where L1+ sizes are smaller than L0. This + // causes compaction scoring, which depends on level sizes, to favor L1+ + // at the expense of L0, which may fill up and stall. + ctx.level_max_bytes[i] = std::cmp::max(level_size, base_bytes_max); + level_size = (level_size as f64 * level_multiplier) as u64; + } + ctx + } + + fn get_priority_levels(&self, levels: &Levels, handlers: &[LevelHandler]) -> SelectContext { + let mut ctx = self.calculate_level_base_size(levels); + + let idle_file_count = levels + .l0 + .as_ref() + .unwrap() + .sub_levels + .iter() + .map(|level| level.table_infos.len()) + .sum::() + - handlers[0].get_pending_file_count(); + + if idle_file_count > 0 { + // trigger l0 compaction when the number of files is too large. + + // The read query at the overlapping level needs to merge all the ssts, so the number of + // ssts is the most important factor affecting the read performance, we use file count + // to calculate the score + let overlapping_file_count = levels + .l0 + .as_ref() + .unwrap() + .sub_levels + .iter() + .filter(|level| level.level_type() == LevelType::Overlapping) + .map(|level| level.table_infos.len()) + .sum::(); + if overlapping_file_count > 0 { + // FIXME: use overlapping idle file count + let l0_overlapping_score = + std::cmp::min(idle_file_count, overlapping_file_count) as u64 * SCORE_BASE + / self.config.level0_tier_compact_file_number; + // Reduce the level num of l0 overlapping sub_level + ctx.score_levels.push(PickerInfo { + score: std::cmp::max(l0_overlapping_score, SCORE_BASE + 1), + select_level: 0, + target_level: 0, + picker_type: PickerType::Tier, + }) + } + + // The read query at the non-overlapping level only selects ssts that match the query + // range at each level, so the number of levels is the most important factor affecting + // the read performance. At the same time, the size factor is also added to the score + // calculation rule to avoid unbalanced compact task due to large size. + let non_overlapping_score = { + let total_size = levels.l0.as_ref().unwrap().total_file_size + - handlers[0].get_pending_output_file_size(ctx.base_level as u32); + let base_level_size = levels.get_level(ctx.base_level).total_file_size; + let base_level_sst_count = + levels.get_level(ctx.base_level).table_infos.len() as u64; + + // size limit + let non_overlapping_size_score = total_size * SCORE_BASE + / std::cmp::max(self.config.max_bytes_for_level_base, base_level_size); + // level count limit + let non_overlapping_level_count = levels + .l0 + .as_ref() + .unwrap() + .sub_levels + .iter() + .filter(|level| level.level_type() == LevelType::Nonoverlapping) + .count() as u64; + let non_overlapping_level_score = non_overlapping_level_count * SCORE_BASE + / std::cmp::max( + base_level_sst_count / 16, + self.config.level0_sub_level_compact_level_count as u64, + ); + + std::cmp::max(non_overlapping_size_score, non_overlapping_level_score) + }; + + // Reduce the level num of l0 non-overlapping sub_level + ctx.score_levels.push({ + PickerInfo { + score: non_overlapping_score, + select_level: 0, + target_level: ctx.base_level, + picker_type: PickerType::ToBase, + } + }); + + // FIXME: more accurate score calculation algorithm will be introduced (#11903) + ctx.score_levels.push({ + PickerInfo { + score: non_overlapping_score, + select_level: 0, + target_level: 0, + picker_type: PickerType::Intra, + } + }); + } + + // The bottommost level can not be input level. + for level in &levels.levels { + let level_idx = level.level_idx as usize; + if level_idx < ctx.base_level || level_idx >= self.config.max_level as usize { + continue; + } + let upper_level = if level_idx == ctx.base_level { + 0 + } else { + level_idx - 1 + }; + let total_size = level.total_file_size + + handlers[upper_level].get_pending_output_file_size(level.level_idx) + - handlers[level_idx].get_pending_output_file_size(level.level_idx + 1); + if total_size == 0 { + continue; + } + + ctx.score_levels.push({ + PickerInfo { + score: total_size * SCORE_BASE / ctx.level_max_bytes[level_idx], + select_level: level_idx, + target_level: level_idx + 1, + picker_type: PickerType::BottomLevel, + } + }); + } + + // sort reverse to pick the largest one. + ctx.score_levels.sort_by(|a, b| { + b.score + .cmp(&a.score) + .then_with(|| a.target_level.cmp(&b.target_level)) + }); + ctx + } + + /// `compact_pending_bytes_needed` calculates the number of compact bytes needed to balance the + /// LSM Tree from the current state of each level in the LSM Tree in combination with + /// `compaction_config` + /// This algorithm refers to the implementation in [`https://github.com/facebook/rocksdb/blob/main/db/version_set.cc#L3141`] + pub fn compact_pending_bytes_needed(&self, levels: &Levels) -> u64 { + let ctx = self.calculate_level_base_size(levels); + self.compact_pending_bytes_needed_with_ctx(levels, &ctx) + } + + pub fn compact_pending_bytes_needed_with_ctx( + &self, + levels: &Levels, + ctx: &SelectContext, + ) -> u64 { + // l0 + let mut compact_pending_bytes = 0; + let mut compact_to_next_level_bytes = 0; + let l0_size = levels + .l0 + .as_ref() + .unwrap() + .sub_levels + .iter() + .map(|sub_level| sub_level.total_file_size) + .sum::(); + + let mut l0_compaction_trigger = false; + if l0_size > self.config.max_bytes_for_level_base { + compact_pending_bytes = l0_size; + compact_to_next_level_bytes = l0_size; + l0_compaction_trigger = true; + } + + // l1 and up + let mut level_bytes; + let mut next_level_bytes = 0; + for level in &levels.levels[ctx.base_level - 1..levels.levels.len()] { + let level_index = level.get_level_idx() as usize; + + if next_level_bytes > 0 { + level_bytes = next_level_bytes; + next_level_bytes = 0; + } else { + level_bytes = level.total_file_size; + } + + if level_index == ctx.base_level && l0_compaction_trigger { + compact_pending_bytes += level_bytes; + } + + level_bytes += compact_to_next_level_bytes; + compact_to_next_level_bytes = 0; + let level_target = ctx.level_max_bytes[level_index]; + if level_bytes > level_target { + compact_to_next_level_bytes = level_bytes - level_target; + + // Estimate the actual compaction fan-out ratio as size ratio between + // the two levels. + assert_eq!(0, next_level_bytes); + if level_index + 1 < ctx.level_max_bytes.len() { + let next_level = level_index + 1; + next_level_bytes = levels.levels[next_level - 1].total_file_size; + } + + if next_level_bytes > 0 { + compact_pending_bytes += (compact_to_next_level_bytes as f64 + * (next_level_bytes as f64 / level_bytes as f64 + 1.0)) + as u64; + } + } + } + + compact_pending_bytes + } +} + +impl CompactionSelector for DynamicLevelSelector { + fn pick_compaction( + &mut self, + task_id: HummockCompactionTaskId, + compaction_group: &CompactionGroup, + levels: &Levels, + level_handlers: &mut [LevelHandler], + selector_stats: &mut LocalSelectorStatistic, + _table_id_to_options: HashMap, + ) -> Option { + let dynamic_level_core = + DynamicLevelSelectorCore::new(compaction_group.compaction_config.clone()); + let overlap_strategy = + create_overlap_strategy(compaction_group.compaction_config.compaction_mode()); + let ctx = dynamic_level_core.get_priority_levels(levels, level_handlers); + // TODO: Determine which rule to enable by write limit + let compaction_task_validator = Arc::new(CompactionTaskValidator::new( + compaction_group.compaction_config.clone(), + )); + for picker_info in &ctx.score_levels { + if picker_info.score <= SCORE_BASE { + return None; + } + let mut picker = dynamic_level_core.create_compaction_picker( + picker_info, + overlap_strategy.clone(), + compaction_task_validator.clone(), + ); + + let mut stats = LocalPickerStatistic::default(); + if let Some(ret) = picker.pick_compaction(levels, level_handlers, &mut stats) { + ret.add_pending_task(task_id, level_handlers); + return Some(create_compaction_task( + dynamic_level_core.get_config(), + ret, + ctx.base_level, + self.task_type(), + )); + } + selector_stats.skip_picker.push(( + picker_info.select_level, + picker_info.target_level, + stats, + )); + } + None + } + + fn name(&self) -> &'static str { + "DynamicLevelSelector" + } + + fn task_type(&self) -> compact_task::TaskType { + compact_task::TaskType::Dynamic + } +} + +#[cfg(test)] +pub mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use itertools::Itertools; + use risingwave_common::constants::hummock::CompactionFilterFlag; + use risingwave_pb::hummock::compaction_config::CompactionMode; + use risingwave_pb::hummock::hummock_version::Levels; + + use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; + use crate::hummock::compaction::selector::tests::{ + assert_compaction_task, generate_l0_nonoverlapping_sublevels, generate_level, + generate_tables, push_tables_level0_nonoverlapping, + }; + use crate::hummock::compaction::selector::{ + CompactionSelector, DynamicLevelSelector, DynamicLevelSelectorCore, LocalSelectorStatistic, + }; + use crate::hummock::level_handler::LevelHandler; + use crate::hummock::model::CompactionGroup; + + #[test] + fn test_dynamic_level() { + let config = CompactionConfigBuilder::new() + .max_bytes_for_level_base(100) + .max_level(4) + .max_bytes_for_level_multiplier(5) + .max_compaction_bytes(1) + .level0_tier_compact_file_number(2) + .compaction_mode(CompactionMode::Range as i32) + .build(); + let selector = DynamicLevelSelectorCore::new(Arc::new(config)); + let levels = vec![ + generate_level(1, vec![]), + generate_level(2, generate_tables(0..5, 0..1000, 3, 10)), + generate_level(3, generate_tables(5..10, 0..1000, 2, 50)), + generate_level(4, generate_tables(10..15, 0..1000, 1, 200)), + ]; + let mut levels = Levels { + levels, + l0: Some(generate_l0_nonoverlapping_sublevels(vec![])), + ..Default::default() + }; + let ctx = selector.calculate_level_base_size(&levels); + assert_eq!(ctx.base_level, 2); + assert_eq!(ctx.level_max_bytes[2], 100); + assert_eq!(ctx.level_max_bytes[3], 200); + assert_eq!(ctx.level_max_bytes[4], 1000); + + levels.levels[3] + .table_infos + .append(&mut generate_tables(15..20, 2000..3000, 1, 400)); + levels.levels[3].total_file_size = levels.levels[3] + .table_infos + .iter() + .map(|sst| sst.file_size) + .sum::(); + + let ctx = selector.calculate_level_base_size(&levels); + // data size increase, so we need increase one level to place more data. + assert_eq!(ctx.base_level, 1); + assert_eq!(ctx.level_max_bytes[1], 100); + assert_eq!(ctx.level_max_bytes[2], 120); + assert_eq!(ctx.level_max_bytes[3], 600); + assert_eq!(ctx.level_max_bytes[4], 3000); + + // append a large data to L0 but it does not change the base size of LSM tree. + push_tables_level0_nonoverlapping(&mut levels, generate_tables(20..26, 0..1000, 1, 100)); + + let ctx = selector.calculate_level_base_size(&levels); + assert_eq!(ctx.base_level, 1); + assert_eq!(ctx.level_max_bytes[1], 100); + assert_eq!(ctx.level_max_bytes[2], 120); + assert_eq!(ctx.level_max_bytes[3], 600); + assert_eq!(ctx.level_max_bytes[4], 3000); + + levels.l0.as_mut().unwrap().sub_levels.clear(); + levels.l0.as_mut().unwrap().total_file_size = 0; + levels.levels[0].table_infos = generate_tables(26..32, 0..1000, 1, 100); + levels.levels[0].total_file_size = levels.levels[0] + .table_infos + .iter() + .map(|sst| sst.file_size) + .sum::(); + + let ctx = selector.calculate_level_base_size(&levels); + assert_eq!(ctx.base_level, 1); + assert_eq!(ctx.level_max_bytes[1], 100); + assert_eq!(ctx.level_max_bytes[2], 120); + assert_eq!(ctx.level_max_bytes[3], 600); + assert_eq!(ctx.level_max_bytes[4], 3000); + } + + #[test] + fn test_pick_compaction() { + let config = CompactionConfigBuilder::new() + .max_bytes_for_level_base(200) + .max_level(4) + .max_bytes_for_level_multiplier(5) + .target_file_size_base(5) + .max_compaction_bytes(10000) + .level0_tier_compact_file_number(4) + .compaction_mode(CompactionMode::Range as i32) + .level0_sub_level_compact_level_count(1) + .build(); + let group_config = CompactionGroup::new(1, config.clone()); + let levels = vec![ + generate_level(1, vec![]), + generate_level(2, generate_tables(0..5, 0..1000, 3, 10)), + generate_level(3, generate_tables(5..10, 0..1000, 2, 50)), + generate_level(4, generate_tables(10..15, 0..1000, 1, 200)), + ]; + let mut levels = Levels { + levels, + l0: Some(generate_l0_nonoverlapping_sublevels(generate_tables( + 15..25, + 0..600, + 3, + 10, + ))), + member_table_ids: vec![1], + ..Default::default() + }; + + let mut selector = DynamicLevelSelector::default(); + let mut levels_handlers = (0..5).map(LevelHandler::new).collect_vec(); + let mut local_stats = LocalSelectorStatistic::default(); + let compaction = selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handlers, + &mut local_stats, + HashMap::default(), + ) + .unwrap(); + assert_compaction_task(&compaction, &levels_handlers); + + let compaction_filter_flag = CompactionFilterFlag::STATE_CLEAN | CompactionFilterFlag::TTL; + let config = CompactionConfigBuilder::with_config(config) + .max_bytes_for_level_base(100) + .sub_level_max_compaction_bytes(50) + .target_file_size_base(20) + .compaction_filter_mask(compaction_filter_flag.into()) + .build(); + let group_config = CompactionGroup::new(1, config.clone()); + let mut selector = DynamicLevelSelector::default(); + + levels.l0.as_mut().unwrap().sub_levels.clear(); + levels.l0.as_mut().unwrap().total_file_size = 0; + push_tables_level0_nonoverlapping(&mut levels, generate_tables(15..25, 0..600, 3, 20)); + let mut levels_handlers = (0..5).map(LevelHandler::new).collect_vec(); + let compaction = selector + .pick_compaction( + 1, + &group_config, + &levels, + &mut levels_handlers, + &mut local_stats, + HashMap::default(), + ) + .unwrap(); + assert_compaction_task(&compaction, &levels_handlers); + assert_eq!(compaction.input.input_levels[0].level_idx, 0); + assert_eq!(compaction.input.target_level, 2); + + levels_handlers[0].remove_task(1); + levels_handlers[2].remove_task(1); + levels.l0.as_mut().unwrap().sub_levels.clear(); + levels.levels[1].table_infos = generate_tables(20..30, 0..1000, 3, 10); + let compaction = selector + .pick_compaction( + 2, + &group_config, + &levels, + &mut levels_handlers, + &mut local_stats, + HashMap::default(), + ) + .unwrap(); + assert_compaction_task(&compaction, &levels_handlers); + assert_eq!(compaction.input.input_levels[0].level_idx, 3); + assert_eq!(compaction.input.target_level, 4); + assert_eq!( + compaction.input.input_levels[0] + .table_infos + .iter() + .map(|sst| sst.get_sst_id()) + .collect_vec(), + vec![5] + ); + assert_eq!( + compaction.input.input_levels[1] + .table_infos + .iter() + .map(|sst| sst.get_sst_id()) + .collect_vec(), + vec![10] + ); + assert_eq!( + compaction.target_file_size, + config.target_file_size_base * 2 + ); + assert_eq!(compaction.compression_algorithm.as_str(), "Lz4",); + // no compaction need to be scheduled because we do not calculate the size of pending files + // to score. + let compaction = selector.pick_compaction( + 2, + &group_config, + &levels, + &mut levels_handlers, + &mut local_stats, + HashMap::default(), + ); + assert!(compaction.is_none()); + } + + #[test] + fn test_compact_pending_bytes() { + let config = CompactionConfigBuilder::new() + .max_bytes_for_level_base(100) + .max_level(4) + .max_bytes_for_level_multiplier(5) + .compaction_mode(CompactionMode::Range as i32) + .build(); + let levels = vec![ + generate_level(1, vec![]), + generate_level(2, generate_tables(0..50, 0..1000, 3, 500)), + generate_level(3, generate_tables(30..60, 0..1000, 2, 500)), + generate_level(4, generate_tables(60..70, 0..1000, 1, 1000)), + ]; + let levels = Levels { + levels, + l0: Some(generate_l0_nonoverlapping_sublevels(generate_tables( + 15..25, + 0..600, + 3, + 100, + ))), + ..Default::default() + }; + + let dynamic_level_core = DynamicLevelSelectorCore::new(Arc::new(config)); + let ctx = dynamic_level_core.calculate_level_base_size(&levels); + assert_eq!(1, ctx.base_level); + assert_eq!(1000, levels.l0.as_ref().unwrap().total_file_size); // l0 + assert_eq!(0, levels.levels.get(0).unwrap().total_file_size); // l1 + assert_eq!(25000, levels.levels.get(1).unwrap().total_file_size); // l2 + assert_eq!(15000, levels.levels.get(2).unwrap().total_file_size); // l3 + assert_eq!(10000, levels.levels.get(3).unwrap().total_file_size); // l4 + + assert_eq!(100, ctx.level_max_bytes[1]); // l1 + assert_eq!(500, ctx.level_max_bytes[2]); // l2 + assert_eq!(2500, ctx.level_max_bytes[3]); // l3 + assert_eq!(12500, ctx.level_max_bytes[4]); // l4 + + // l1 pending = (0 + 1000 - 100) * ((25000 / 1000) + 1) + 1000 = 24400 + // l2 pending = (25000 + 900 - 500) * ((15000 / (25000 + 900)) + 1) = 40110 + // l3 pending = (15000 + 25400 - 2500) * ((10000 / (15000 + 25400) + 1)) = 47281 + + let compact_pending_bytes = dynamic_level_core.compact_pending_bytes_needed(&levels); + assert_eq!(24400 + 40110 + 47281, compact_pending_bytes); + } +} diff --git a/src/meta/src/hummock/compaction/selector/manual_selector.rs b/src/meta/src/hummock/compaction/selector/manual_selector.rs new file mode 100644 index 0000000000000..a00565a9807cd --- /dev/null +++ b/src/meta/src/hummock/compaction/selector/manual_selector.rs @@ -0,0 +1,122 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +use std::collections::{HashMap, HashSet}; + +use risingwave_common::catalog::TableOption; +use risingwave_hummock_sdk::compaction_group::StateTableId; +use risingwave_hummock_sdk::{HummockCompactionTaskId, HummockSstableId}; +use risingwave_pb::hummock::hummock_version::Levels; +use risingwave_pb::hummock::{compact_task, KeyRange}; + +use super::{CompactionSelector, DynamicLevelSelectorCore, LocalSelectorStatistic}; +use crate::hummock::compaction::picker::{ + CompactionPicker, LocalPickerStatistic, ManualCompactionPicker, +}; +use crate::hummock::compaction::{create_compaction_task, create_overlap_strategy, CompactionTask}; +use crate::hummock::level_handler::LevelHandler; +use crate::hummock::model::CompactionGroup; + +#[derive(Clone, Debug, PartialEq)] +pub struct ManualCompactionOption { + /// Filters out SSTs to pick. Has no effect if empty. + pub sst_ids: Vec, + /// Filters out SSTs to pick. + pub key_range: KeyRange, + /// Filters out SSTs to pick. Has no effect if empty. + pub internal_table_id: HashSet, + /// Input level. + pub level: usize, +} + +impl Default for ManualCompactionOption { + fn default() -> Self { + Self { + sst_ids: vec![], + key_range: KeyRange { + left: vec![], + right: vec![], + right_exclusive: false, + }, + internal_table_id: HashSet::default(), + level: 1, + } + } +} + +pub struct ManualCompactionSelector { + option: ManualCompactionOption, +} + +impl ManualCompactionSelector { + pub fn new(option: ManualCompactionOption) -> Self { + Self { option } + } +} + +impl CompactionSelector for ManualCompactionSelector { + fn pick_compaction( + &mut self, + task_id: HummockCompactionTaskId, + group: &CompactionGroup, + levels: &Levels, + level_handlers: &mut [LevelHandler], + _selector_stats: &mut LocalSelectorStatistic, + _table_id_to_options: HashMap, + ) -> Option { + let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); + let overlap_strategy = create_overlap_strategy(group.compaction_config.compaction_mode()); + let ctx = dynamic_level_core.calculate_level_base_size(levels); + let (mut picker, base_level) = { + let target_level = if self.option.level == 0 { + ctx.base_level + } else if self.option.level == group.compaction_config.max_level as usize { + self.option.level + } else { + self.option.level + 1 + }; + if self.option.level > 0 && self.option.level < ctx.base_level { + return None; + } + ( + ManualCompactionPicker::new(overlap_strategy, self.option.clone(), target_level), + ctx.base_level, + ) + }; + + let compaction_input = + picker.pick_compaction(levels, level_handlers, &mut LocalPickerStatistic::default())?; + compaction_input.add_pending_task(task_id, level_handlers); + + Some(create_compaction_task( + group.compaction_config.as_ref(), + compaction_input, + base_level, + self.task_type(), + )) + } + + fn name(&self) -> &'static str { + "ManualCompactionSelector" + } + + fn task_type(&self) -> compact_task::TaskType { + compact_task::TaskType::Manual + } +} diff --git a/src/meta/src/hummock/compaction/selector/mod.rs b/src/meta/src/hummock/compaction/selector/mod.rs new file mode 100644 index 0000000000000..1bc49afc55653 --- /dev/null +++ b/src/meta/src/hummock/compaction/selector/mod.rs @@ -0,0 +1,348 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +mod emergency_selector; +mod level_selector; +mod manual_selector; +mod space_reclaim_selector; +mod tombstone_compaction_selector; +mod ttl_selector; + +use std::collections::HashMap; + +pub use emergency_selector::EmergencySelector; +pub use level_selector::{DynamicLevelSelector, DynamicLevelSelectorCore}; +pub use manual_selector::{ManualCompactionOption, ManualCompactionSelector}; +use risingwave_common::catalog::TableOption; +use risingwave_hummock_sdk::HummockCompactionTaskId; +use risingwave_pb::hummock::compact_task; +use risingwave_pb::hummock::hummock_version::Levels; +pub use space_reclaim_selector::SpaceReclaimCompactionSelector; +pub use tombstone_compaction_selector::TombstoneCompactionSelector; +pub use ttl_selector::TtlCompactionSelector; + +use super::picker::LocalPickerStatistic; +use super::{create_compaction_task, LevelCompactionPicker, TierCompactionPicker}; +use crate::hummock::compaction::CompactionTask; +use crate::hummock::level_handler::LevelHandler; +use crate::hummock::model::CompactionGroup; +use crate::rpc::metrics::MetaMetrics; + +pub trait CompactionSelector: Sync + Send { + fn pick_compaction( + &mut self, + task_id: HummockCompactionTaskId, + group: &CompactionGroup, + levels: &Levels, + level_handlers: &mut [LevelHandler], + selector_stats: &mut LocalSelectorStatistic, + table_id_to_options: HashMap, + ) -> Option; + + fn report_statistic_metrics(&self, _metrics: &MetaMetrics) {} + + fn name(&self) -> &'static str; + + fn task_type(&self) -> compact_task::TaskType; +} + +pub fn default_compaction_selector() -> Box { + Box::::default() +} + +#[derive(Default)] +pub struct LocalSelectorStatistic { + skip_picker: Vec<(usize, usize, LocalPickerStatistic)>, +} + +impl LocalSelectorStatistic { + pub fn report_to_metrics(&self, group_id: u64, metrics: &MetaMetrics) { + for (start_level, target_level, stats) in &self.skip_picker { + let level_label = format!("cg{}-{}-to-{}", group_id, start_level, target_level); + if stats.skip_by_write_amp_limit > 0 { + metrics + .compact_skip_frequency + .with_label_values(&[level_label.as_str(), "write-amp"]) + .inc(); + } + if stats.skip_by_count_limit > 0 { + metrics + .compact_skip_frequency + .with_label_values(&[level_label.as_str(), "count"]) + .inc(); + } + if stats.skip_by_pending_files > 0 { + metrics + .compact_skip_frequency + .with_label_values(&[level_label.as_str(), "pending-files"]) + .inc(); + } + if stats.skip_by_overlapping > 0 { + metrics + .compact_skip_frequency + .with_label_values(&[level_label.as_str(), "overlapping"]) + .inc(); + } + metrics + .compact_skip_frequency + .with_label_values(&[level_label.as_str(), "picker"]) + .inc(); + } + } +} + +#[cfg(test)] +pub mod tests { + use std::ops::Range; + + use itertools::Itertools; + use risingwave_pb::hummock::{KeyRange, Level, LevelType, OverlappingLevel, SstableInfo}; + + use super::*; + use crate::hummock::test_utils::iterator_test_key_of_epoch; + + pub fn push_table_level0_overlapping(levels: &mut Levels, sst: SstableInfo) { + levels.l0.as_mut().unwrap().total_file_size += sst.file_size; + levels.l0.as_mut().unwrap().sub_levels.push(Level { + level_idx: 0, + level_type: LevelType::Overlapping as i32, + total_file_size: sst.file_size, + uncompressed_file_size: sst.uncompressed_file_size, + sub_level_id: sst.get_sst_id(), + table_infos: vec![sst], + }); + } + + pub fn push_table_level0_nonoverlapping(levels: &mut Levels, sst: SstableInfo) { + push_table_level0_overlapping(levels, sst); + levels + .l0 + .as_mut() + .unwrap() + .sub_levels + .last_mut() + .unwrap() + .level_type = LevelType::Nonoverlapping as i32; + } + + pub fn push_tables_level0_nonoverlapping(levels: &mut Levels, table_infos: Vec) { + let total_file_size = table_infos.iter().map(|table| table.file_size).sum::(); + let uncompressed_file_size = table_infos + .iter() + .map(|table| table.uncompressed_file_size) + .sum(); + let sub_level_id = table_infos[0].get_sst_id(); + levels.l0.as_mut().unwrap().total_file_size += total_file_size; + levels.l0.as_mut().unwrap().sub_levels.push(Level { + level_idx: 0, + level_type: LevelType::Nonoverlapping as i32, + total_file_size, + sub_level_id, + table_infos, + uncompressed_file_size, + }); + } + + pub fn generate_table( + id: u64, + table_prefix: u64, + left: usize, + right: usize, + epoch: u64, + ) -> SstableInfo { + SstableInfo { + object_id: id, + sst_id: id, + key_range: Some(KeyRange { + left: iterator_test_key_of_epoch(table_prefix, left, epoch), + right: iterator_test_key_of_epoch(table_prefix, right, epoch), + right_exclusive: false, + }), + file_size: (right - left + 1) as u64, + table_ids: vec![table_prefix as u32], + uncompressed_file_size: (right - left + 1) as u64, + total_key_count: (right - left + 1) as u64, + ..Default::default() + } + } + + #[allow(clippy::too_many_arguments)] + pub fn generate_table_with_ids_and_epochs( + id: u64, + table_prefix: u64, + left: usize, + right: usize, + epoch: u64, + table_ids: Vec, + min_epoch: u64, + max_epoch: u64, + ) -> SstableInfo { + SstableInfo { + object_id: id, + sst_id: id, + key_range: Some(KeyRange { + left: iterator_test_key_of_epoch(table_prefix, left, epoch), + right: iterator_test_key_of_epoch(table_prefix, right, epoch), + right_exclusive: false, + }), + file_size: (right - left + 1) as u64, + table_ids, + uncompressed_file_size: (right - left + 1) as u64, + min_epoch, + max_epoch, + ..Default::default() + } + } + + pub fn generate_tables( + ids: Range, + keys: Range, + epoch: u64, + file_size: u64, + ) -> Vec { + let step = (keys.end - keys.start) / (ids.end - ids.start) as usize; + let mut start = keys.start; + let mut tables = vec![]; + for id in ids { + let mut table = generate_table(id, 1, start, start + step - 1, epoch); + table.file_size = file_size; + tables.push(table); + start += step; + } + tables + } + + pub fn generate_level(level_idx: u32, table_infos: Vec) -> Level { + let total_file_size = table_infos.iter().map(|sst| sst.file_size).sum(); + let uncompressed_file_size = table_infos + .iter() + .map(|sst| sst.uncompressed_file_size) + .sum(); + Level { + level_idx, + level_type: LevelType::Nonoverlapping as i32, + table_infos, + total_file_size, + sub_level_id: 0, + uncompressed_file_size, + } + } + + /// Returns a `OverlappingLevel`, with each `table_infos`'s element placed in a nonoverlapping + /// sub-level. + pub fn generate_l0_nonoverlapping_sublevels(table_infos: Vec) -> OverlappingLevel { + let total_file_size = table_infos.iter().map(|table| table.file_size).sum::(); + let uncompressed_file_size = table_infos + .iter() + .map(|table| table.uncompressed_file_size) + .sum::(); + OverlappingLevel { + sub_levels: table_infos + .into_iter() + .enumerate() + .map(|(idx, table)| Level { + level_idx: 0, + level_type: LevelType::Nonoverlapping as i32, + total_file_size: table.file_size, + uncompressed_file_size: table.uncompressed_file_size, + sub_level_id: idx as u64, + table_infos: vec![table], + }) + .collect_vec(), + total_file_size, + uncompressed_file_size, + } + } + + pub fn generate_l0_nonoverlapping_multi_sublevels( + table_infos: Vec>, + ) -> OverlappingLevel { + let mut l0 = OverlappingLevel { + sub_levels: table_infos + .into_iter() + .enumerate() + .map(|(idx, table)| Level { + level_idx: 0, + level_type: LevelType::Nonoverlapping as i32, + total_file_size: table.iter().map(|table| table.file_size).sum::(), + uncompressed_file_size: table + .iter() + .map(|sst| sst.uncompressed_file_size) + .sum::(), + sub_level_id: idx as u64, + table_infos: table, + }) + .collect_vec(), + total_file_size: 0, + uncompressed_file_size: 0, + }; + + l0.total_file_size = l0.sub_levels.iter().map(|l| l.total_file_size).sum::(); + l0.uncompressed_file_size = l0 + .sub_levels + .iter() + .map(|l| l.uncompressed_file_size) + .sum::(); + l0 + } + + /// Returns a `OverlappingLevel`, with each `table_infos`'s element placed in a overlapping + /// sub-level. + pub fn generate_l0_overlapping_sublevels( + table_infos: Vec>, + ) -> OverlappingLevel { + let mut l0 = OverlappingLevel { + sub_levels: table_infos + .into_iter() + .enumerate() + .map(|(idx, table)| Level { + level_idx: 0, + level_type: LevelType::Overlapping as i32, + total_file_size: table.iter().map(|table| table.file_size).sum::(), + sub_level_id: idx as u64, + table_infos: table.clone(), + uncompressed_file_size: table + .iter() + .map(|sst| sst.uncompressed_file_size) + .sum::(), + }) + .collect_vec(), + total_file_size: 0, + uncompressed_file_size: 0, + }; + l0.total_file_size = l0.sub_levels.iter().map(|l| l.total_file_size).sum::(); + l0.uncompressed_file_size = l0 + .sub_levels + .iter() + .map(|l| l.uncompressed_file_size) + .sum::(); + l0 + } + + pub(crate) fn assert_compaction_task( + compact_task: &CompactionTask, + level_handlers: &[LevelHandler], + ) { + for i in &compact_task.input.input_levels { + for t in &i.table_infos { + assert!(level_handlers[i.level_idx as usize].is_pending_compact(&t.sst_id)); + } + } + } +} diff --git a/src/meta/src/hummock/compaction/selector/space_reclaim_selector.rs b/src/meta/src/hummock/compaction/selector/space_reclaim_selector.rs new file mode 100644 index 0000000000000..48941a4273d66 --- /dev/null +++ b/src/meta/src/hummock/compaction/selector/space_reclaim_selector.rs @@ -0,0 +1,74 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +use std::collections::HashMap; + +use risingwave_common::catalog::TableOption; +use risingwave_hummock_sdk::HummockCompactionTaskId; +use risingwave_pb::hummock::compact_task; +use risingwave_pb::hummock::hummock_version::Levels; + +use super::{CompactionSelector, DynamicLevelSelectorCore}; +use crate::hummock::compaction::picker::{SpaceReclaimCompactionPicker, SpaceReclaimPickerState}; +use crate::hummock::compaction::{create_compaction_task, CompactionTask, LocalSelectorStatistic}; +use crate::hummock::level_handler::LevelHandler; +use crate::hummock::model::CompactionGroup; + +#[derive(Default)] +pub struct SpaceReclaimCompactionSelector { + state: HashMap, +} + +impl CompactionSelector for SpaceReclaimCompactionSelector { + fn pick_compaction( + &mut self, + task_id: HummockCompactionTaskId, + group: &CompactionGroup, + levels: &Levels, + level_handlers: &mut [LevelHandler], + _selector_stats: &mut LocalSelectorStatistic, + _table_id_to_options: HashMap, + ) -> Option { + let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); + let mut picker = SpaceReclaimCompactionPicker::new( + group.compaction_config.max_space_reclaim_bytes, + levels.member_table_ids.iter().cloned().collect(), + ); + let ctx = dynamic_level_core.calculate_level_base_size(levels); + let state = self.state.entry(group.group_id).or_default(); + + let compaction_input = picker.pick_compaction(levels, level_handlers, state)?; + compaction_input.add_pending_task(task_id, level_handlers); + + Some(create_compaction_task( + dynamic_level_core.get_config(), + compaction_input, + ctx.base_level, + self.task_type(), + )) + } + + fn name(&self) -> &'static str { + "SpaceReclaimCompaction" + } + + fn task_type(&self) -> compact_task::TaskType { + compact_task::TaskType::SpaceReclaim + } +} diff --git a/src/meta/src/hummock/compaction/tombstone_compaction_selector.rs b/src/meta/src/hummock/compaction/selector/tombstone_compaction_selector.rs similarity index 94% rename from src/meta/src/hummock/compaction/tombstone_compaction_selector.rs rename to src/meta/src/hummock/compaction/selector/tombstone_compaction_selector.rs index f6a26dcc13013..49b333403e8ab 100644 --- a/src/meta/src/hummock/compaction/tombstone_compaction_selector.rs +++ b/src/meta/src/hummock/compaction/selector/tombstone_compaction_selector.rs @@ -19,12 +19,12 @@ use risingwave_hummock_sdk::HummockCompactionTaskId; use risingwave_pb::hummock::compact_task; use risingwave_pb::hummock::hummock_version::Levels; +use super::{CompactionSelector, DynamicLevelSelectorCore}; use crate::hummock::compaction::picker::{ TombstoneReclaimCompactionPicker, TombstoneReclaimPickerState, }; use crate::hummock::compaction::{ - create_compaction_task, create_overlap_strategy, CompactionTask, DynamicLevelSelectorCore, - LevelSelector, LocalSelectorStatistic, + create_compaction_task, create_overlap_strategy, CompactionTask, LocalSelectorStatistic, }; use crate::hummock::level_handler::LevelHandler; use crate::hummock::model::CompactionGroup; @@ -34,7 +34,7 @@ pub struct TombstoneCompactionSelector { state: HashMap, } -impl LevelSelector for TombstoneCompactionSelector { +impl CompactionSelector for TombstoneCompactionSelector { fn pick_compaction( &mut self, task_id: HummockCompactionTaskId, diff --git a/src/meta/src/hummock/compaction/selector/ttl_selector.rs b/src/meta/src/hummock/compaction/selector/ttl_selector.rs new file mode 100644 index 0000000000000..920bf36883d78 --- /dev/null +++ b/src/meta/src/hummock/compaction/selector/ttl_selector.rs @@ -0,0 +1,73 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +use std::collections::HashMap; + +use risingwave_common::catalog::TableOption; +use risingwave_hummock_sdk::HummockCompactionTaskId; +use risingwave_pb::hummock::compact_task; +use risingwave_pb::hummock::hummock_version::Levels; + +use super::{CompactionSelector, DynamicLevelSelectorCore}; +use crate::hummock::compaction::picker::{TtlPickerState, TtlReclaimCompactionPicker}; +use crate::hummock::compaction::{create_compaction_task, CompactionTask, LocalSelectorStatistic}; +use crate::hummock::level_handler::LevelHandler; +use crate::hummock::model::CompactionGroup; + +#[derive(Default)] +pub struct TtlCompactionSelector { + state: HashMap, +} + +impl CompactionSelector for TtlCompactionSelector { + fn pick_compaction( + &mut self, + task_id: HummockCompactionTaskId, + group: &CompactionGroup, + levels: &Levels, + level_handlers: &mut [LevelHandler], + _selector_stats: &mut LocalSelectorStatistic, + table_id_to_options: HashMap, + ) -> Option { + let dynamic_level_core = DynamicLevelSelectorCore::new(group.compaction_config.clone()); + let ctx = dynamic_level_core.calculate_level_base_size(levels); + let picker = TtlReclaimCompactionPicker::new( + group.compaction_config.max_space_reclaim_bytes, + table_id_to_options, + ); + let state = self.state.entry(group.group_id).or_default(); + let compaction_input = picker.pick_compaction(levels, level_handlers, state)?; + compaction_input.add_pending_task(task_id, level_handlers); + + Some(create_compaction_task( + group.compaction_config.as_ref(), + compaction_input, + ctx.base_level, + self.task_type(), + )) + } + + fn name(&self) -> &'static str { + "TtlCompaction" + } + + fn task_type(&self) -> compact_task::TaskType { + compact_task::TaskType::Ttl + } +} diff --git a/src/meta/src/hummock/compactor_manager.rs b/src/meta/src/hummock/compactor_manager.rs index c6dd5e2f82387..c3e62d98a190f 100644 --- a/src/meta/src/hummock/compactor_manager.rs +++ b/src/meta/src/hummock/compactor_manager.rs @@ -453,7 +453,7 @@ mod tests { use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_pb::hummock::CompactTaskProgress; - use crate::hummock::compaction::default_level_selector; + use crate::hummock::compaction::selector::default_compaction_selector; use crate::hummock::test_utils::{ add_ssts, register_table_ids_to_compaction_group, setup_compute_env, }; @@ -477,7 +477,7 @@ mod tests { hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 5effaee29077b..26a27674140c6 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -65,9 +65,11 @@ use tokio_stream::wrappers::IntervalStream; use tonic::Streaming; use tracing::warn; -use crate::hummock::compaction::{ - CompactStatus, LocalSelectorStatistic, ManualCompactionOption, TombstoneCompactionSelector, +use crate::hummock::compaction::selector::{ + DynamicLevelSelector, LocalSelectorStatistic, ManualCompactionOption, ManualCompactionSelector, + SpaceReclaimCompactionSelector, TombstoneCompactionSelector, TtlCompactionSelector, }; +use crate::hummock::compaction::CompactStatus; use crate::hummock::error::{Error, Result}; use crate::hummock::metrics_utils::{ trigger_delta_log_stats, trigger_lsm_stat, trigger_mv_stat, trigger_pin_unpin_snapshot_state, @@ -224,10 +226,6 @@ macro_rules! start_measure_real_process_timer { } pub(crate) use start_measure_real_process_timer; -use super::compaction::{ - DynamicLevelSelector, EmergencySelector, LevelSelector, ManualCompactionSelector, - SpaceReclaimCompactionSelector, TtlCompactionSelector, -}; use crate::hummock::manager::compaction_group_manager::CompactionGroupManager; use crate::hummock::manager::worker::HummockManagerEventSender; @@ -776,7 +774,7 @@ impl HummockManager { pub async fn get_compact_task_impl( &self, compaction_group_id: CompactionGroupId, - selector: &mut Box, + selector: &mut Box, ) -> Result> { // TODO: `get_all_table_options` will hold catalog_manager async lock, to avoid holding the // lock in compaction_guard, take out all table_options in advance there may be a @@ -1078,7 +1076,7 @@ impl HummockManager { pub async fn get_compact_task( &self, compaction_group_id: CompactionGroupId, - selector: &mut Box, + selector: &mut Box, ) -> Result> { fail_point!("fp_get_compact_task", |_| Err(Error::MetaStore( anyhow::anyhow!("failpoint metastore error") @@ -1105,7 +1103,7 @@ impl HummockManager { compaction_group_id: CompactionGroupId, manual_compaction_option: ManualCompactionOption, ) -> Result> { - let mut selector: Box = + let mut selector: Box = Box::new(ManualCompactionSelector::new(manual_compaction_option)); self.get_compact_task(compaction_group_id, &mut selector) .await @@ -2618,7 +2616,7 @@ impl HummockManager { assert_ne!(0, pull_task_count); if let Some(compactor) = hummock_manager.compactor_manager.get_compactor(context_id) { if let Some((group, task_type)) = hummock_manager.auto_pick_compaction_group_and_type().await { - let selector: &mut Box = { + let selector: &mut Box = { let versioning_guard = read_lock!(hummock_manager, versioning).await; let versioning = versioning_guard.deref(); @@ -2945,8 +2943,8 @@ async fn write_exclusive_cluster_id( } } -fn init_selectors() -> HashMap> { - let mut compaction_selectors: HashMap> = +fn init_selectors() -> HashMap> { + let mut compaction_selectors: HashMap> = HashMap::default(); compaction_selectors.insert( compact_task::TaskType::Dynamic, @@ -2974,6 +2972,9 @@ fn init_selectors() -> HashMap> { type CompactionRequestChannelItem = (CompactionGroupId, compact_task::TaskType); use tokio::sync::mpsc::error::SendError; +use super::compaction::selector::EmergencySelector; +use super::compaction::CompactionSelector; + #[derive(Debug, Default)] pub struct CompactionState { scheduled: Mutex>, diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 44c90c7855ec9..4d5de0cc19011 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -39,10 +39,11 @@ use risingwave_pb::hummock::{ use risingwave_pb::meta::add_worker_node_request::Property; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; -use crate::hummock::compaction::{ - default_level_selector, CompactStatus, LevelSelector, ManualCompactionOption, +use crate::hummock::compaction::selector::{ + default_compaction_selector, CompactionSelector, ManualCompactionOption, SpaceReclaimCompactionSelector, }; +use crate::hummock::compaction::CompactStatus; use crate::hummock::error::Error; use crate::hummock::test_utils::*; use crate::hummock::{HummockManager, HummockManagerRef}; @@ -161,7 +162,7 @@ async fn test_hummock_compaction_task() { assert!(hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -192,7 +193,7 @@ async fn test_hummock_compaction_task() { let compact_task = hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -217,7 +218,7 @@ async fn test_hummock_compaction_task() { let compact_task = hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -730,7 +731,7 @@ async fn test_print_compact_task() { let compact_task = hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -877,7 +878,7 @@ async fn test_hummock_compaction_task_heartbeat() { assert!(hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -908,7 +909,7 @@ async fn test_hummock_compaction_task_heartbeat() { let compact_task = hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -950,7 +951,7 @@ async fn test_hummock_compaction_task_heartbeat() { let compact_task = hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -997,7 +998,7 @@ async fn test_hummock_compaction_task_heartbeat_removal_on_node_removal() { assert!(hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -1028,7 +1029,7 @@ async fn test_hummock_compaction_task_heartbeat_removal_on_node_removal() { let compact_task = hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -1194,7 +1195,7 @@ async fn test_version_stats() { let compact_task = hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -1633,7 +1634,7 @@ async fn test_split_compaction_group_trivial_expired() { .await .unwrap(); let task = hummock_manager - .get_compact_task(2, &mut default_level_selector()) + .get_compact_task(2, &mut default_compaction_selector()) .await .unwrap() .unwrap(); @@ -1642,8 +1643,8 @@ async fn test_split_compaction_group_trivial_expired() { .split_compaction_group(2, &[100]) .await .unwrap(); - - let mut selector: Box = Box::::default(); + let mut selector: Box = + Box::::default(); let reclaim_task = hummock_manager .get_compact_task_impl(2, &mut selector) .await @@ -1669,7 +1670,7 @@ async fn test_split_compaction_group_trivial_expired() { ); let task2 = hummock_manager - .get_compact_task(new_group_id, &mut default_level_selector()) + .get_compact_task(new_group_id, &mut default_compaction_selector()) .await .unwrap() .unwrap(); @@ -2031,7 +2032,8 @@ async fn test_move_tables_between_compaction_group() { let groups = info.keys().sorted().cloned().collect_vec(); assert_eq!(groups, vec![2, new_group_id]); - let mut selector: Box = Box::::default(); + let mut selector: Box = + Box::::default(); let compaction_task = hummock_manager .get_compact_task(2, &mut selector) diff --git a/src/meta/src/hummock/metrics_utils.rs b/src/meta/src/hummock/metrics_utils.rs index 9917d559a0709..6818b7f68570e 100644 --- a/src/meta/src/hummock/metrics_utils.rs +++ b/src/meta/src/hummock/metrics_utils.rs @@ -32,7 +32,8 @@ use risingwave_pb::hummock::{ HummockVersionCheckpoint, HummockVersionStats, LevelType, }; -use super::compaction::{get_compression_algorithm, DynamicLevelSelectorCore}; +use super::compaction::get_compression_algorithm; +use super::compaction::selector::DynamicLevelSelectorCore; use crate::hummock::compaction::CompactStatus; use crate::rpc::metrics::MetaMetrics; diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index 915beee8e3a3f..063556ace1839 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -38,8 +38,8 @@ use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; use tokio::task::JoinHandle; use tokio_stream::wrappers::UnboundedReceiverStream; -use crate::hummock::compaction::{ - default_level_selector, LevelSelector, SpaceReclaimCompactionSelector, +use crate::hummock::compaction::selector::{ + default_compaction_selector, CompactionSelector, SpaceReclaimCompactionSelector, }; use crate::hummock::HummockManager; @@ -81,7 +81,7 @@ impl MockHummockMetaClient { self.hummock_manager .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap_or(None) @@ -244,8 +244,8 @@ impl HummockMetaClient for MockHummockMetaClient { let (group, task_type) = group_and_type.unwrap(); - let mut selector: Box = match task_type { - compact_task::TaskType::Dynamic => default_level_selector(), + let mut selector: Box = match task_type { + compact_task::TaskType::Dynamic => default_compaction_selector(), compact_task::TaskType::SpaceReclaim => { Box::::default() } diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 70c055d387c85..3d42442ae7c67 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -31,7 +31,7 @@ use risingwave_pb::meta::add_worker_node_request::Property; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; #[cfg(test)] -use crate::hummock::compaction::default_level_selector; +use crate::hummock::compaction::selector::default_compaction_selector; use crate::hummock::{CompactorManager, HummockManager, HummockManagerRef}; use crate::manager::{ ClusterManager, ClusterManagerRef, FragmentManager, MetaSrvEnv, META_NODE_ID, @@ -92,7 +92,7 @@ pub async fn add_test_tables( StaticCompactionGroupId::StateDefault.into(), ) .await; - let mut selector = default_level_selector(); + let mut selector = default_compaction_selector(); let mut compact_task = hummock_manager .get_compact_task(StaticCompactionGroupId::StateDefault.into(), &mut selector) .await diff --git a/src/meta/src/rpc/service/hummock_service.rs b/src/meta/src/rpc/service/hummock_service.rs index f670c9cb220a5..3ae90421d2d87 100644 --- a/src/meta/src/rpc/service/hummock_service.rs +++ b/src/meta/src/rpc/service/hummock_service.rs @@ -24,7 +24,7 @@ use risingwave_pb::hummock::version_update_payload::Payload; use risingwave_pb::hummock::*; use tonic::{Request, Response, Status, Streaming}; -use crate::hummock::compaction::ManualCompactionOption; +use crate::hummock::compaction::selector::ManualCompactionOption; use crate::hummock::{HummockManagerRef, VacuumManagerRef}; use crate::manager::FragmentManagerRef; use crate::rpc::service::RwReceiverStream; diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index c90a99f59e03c..50d739c5d1eb9 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -35,7 +35,9 @@ pub(crate) mod tests { use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; use risingwave_meta::hummock::compaction::compaction_config::CompactionConfigBuilder; - use risingwave_meta::hummock::compaction::{default_level_selector, ManualCompactionOption}; + use risingwave_meta::hummock::compaction::selector::{ + default_compaction_selector, ManualCompactionOption, + }; use risingwave_meta::hummock::test_utils::{ register_table_ids_to_compaction_group, setup_compute_env, setup_compute_env_with_config, unregister_table_ids_from_compaction_group, @@ -270,7 +272,7 @@ pub(crate) mod tests { while let Some(mut compact_task) = hummock_manager_ref .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -436,7 +438,7 @@ pub(crate) mod tests { while let Some(compact_task) = hummock_manager_ref .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap() @@ -634,7 +636,7 @@ pub(crate) mod tests { let compact_task = hummock_manager_ref .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap(); @@ -819,7 +821,7 @@ pub(crate) mod tests { let compact_task = hummock_manager_ref .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap(); @@ -1015,7 +1017,7 @@ pub(crate) mod tests { let compact_task = hummock_manager_ref .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap(); @@ -1207,7 +1209,7 @@ pub(crate) mod tests { let compact_task = hummock_manager_ref .get_compact_task( StaticCompactionGroupId::StateDefault.into(), - &mut default_level_selector(), + &mut default_compaction_selector(), ) .await .unwrap(); diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index 2b16383fedbc3..1b7d6d80029a5 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -28,7 +28,7 @@ use risingwave_hummock_sdk::key::{next_key, user_key}; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; use risingwave_hummock_sdk::HummockVersionId; use risingwave_meta::hummock::compaction::compaction_config::CompactionConfigBuilder; -use risingwave_meta::hummock::compaction::ManualCompactionOption; +use risingwave_meta::hummock::compaction::selector::ManualCompactionOption; use risingwave_meta::hummock::test_utils::{setup_compute_env, setup_compute_env_with_config}; use risingwave_meta::hummock::{HummockManagerRef, MockHummockMetaClient}; use risingwave_rpc_client::HummockMetaClient; From 77af093c60f28e8859af0d4365ba37789c71db5f Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Tue, 26 Sep 2023 14:27:17 +0800 Subject: [PATCH 14/17] feat(frontend): add rw_catalog.rw_hummock_sstables (#12532) --- .../src/catalog/system_catalog/mod.rs | 1 + .../rw_catalog/rw_hummock_version.rs | 80 ++++++++++++++++++- 2 files changed, 78 insertions(+), 3 deletions(-) diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 185f8017311c2..cf5752d3f0bdd 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -407,6 +407,7 @@ prepare_sys_catalog! { { BuiltinCatalog::Table(&RW_HUMMOCK_PINNED_SNAPSHOTS), read_hummock_pinned_snapshots await }, { BuiltinCatalog::Table(&RW_HUMMOCK_CURRENT_VERSION), read_hummock_current_version await }, { BuiltinCatalog::Table(&RW_HUMMOCK_CHECKPOINT_VERSION), read_hummock_checkpoint_version await }, + { BuiltinCatalog::Table(&RW_HUMMOCK_SSTABLES), read_hummock_sstables await }, { BuiltinCatalog::Table(&RW_HUMMOCK_VERSION_DELTAS), read_hummock_version_deltas await }, { BuiltinCatalog::Table(&RW_HUMMOCK_BRANCHED_OBJECTS), read_hummock_branched_objects await }, { BuiltinCatalog::Table(&RW_HUMMOCK_COMPACTION_GROUP_CONFIGS), read_hummock_compaction_group_configs await }, 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 f8096b6c3ccbf..97269341d59f3 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 @@ -45,15 +45,51 @@ pub const RW_HUMMOCK_CHECKPOINT_VERSION: BuiltinTable = BuiltinTable { pk: &[], }; +pub const RW_HUMMOCK_SSTABLES: BuiltinTable = BuiltinTable { + name: "rw_hummock_sstables", + schema: RW_CATALOG_SCHEMA_NAME, + columns: &[ + (DataType::Int64, "sstable_id"), + (DataType::Int64, "object_id"), + (DataType::Int64, "compaction_group_id"), + (DataType::Int32, "level_id"), + (DataType::Int64, "sub_level_id"), + (DataType::Int32, "level_type"), + (DataType::Bytea, "key_range_left"), + (DataType::Bytea, "key_range_right"), + (DataType::Boolean, "right_exclusive"), + (DataType::Int64, "file_size"), + (DataType::Int64, "meta_offset"), + (DataType::Int64, "stale_key_count"), + (DataType::Int64, "total_key_count"), + (DataType::Int64, "min_epoch"), + (DataType::Int64, "max_epoch"), + (DataType::Int64, "uncompressed_file_size"), + (DataType::Int64, "range_tombstone_count"), + (DataType::Int32, "bloom_filter_kind"), + (DataType::Jsonb, "table_ids"), + ], + pk: &[0], +}; + impl SysCatalogReaderImpl { pub async fn read_hummock_current_version(&self) -> Result> { let version = self.meta_client.get_hummock_current_version().await?; - Ok(version_to_rows(&remove_key_range_from_version(version))) + Ok(version_to_compaction_group_rows( + &remove_key_range_from_version(version), + )) } pub async fn read_hummock_checkpoint_version(&self) -> Result> { let version = self.meta_client.get_hummock_checkpoint_version().await?; - Ok(version_to_rows(&remove_key_range_from_version(version))) + Ok(version_to_compaction_group_rows( + &remove_key_range_from_version(version), + )) + } + + pub async fn read_hummock_sstables(&self) -> Result> { + let version = self.meta_client.get_hummock_current_version().await?; + Ok(version_to_sstable_rows(version)) } } @@ -73,7 +109,7 @@ fn remove_key_range_from_version(mut version: HummockVersion) -> HummockVersion version } -fn version_to_rows(version: &HummockVersion) -> Vec { +fn version_to_compaction_group_rows(version: &HummockVersion) -> Vec { version .levels .values() @@ -87,3 +123,41 @@ fn version_to_rows(version: &HummockVersion) -> Vec { }) .collect() } + +fn version_to_sstable_rows(version: HummockVersion) -> Vec { + let mut sstables = vec![]; + for cg in version.levels.into_values() { + for level in cg.levels.into_iter().chain(cg.l0.unwrap().sub_levels) { + for sst in level.table_infos { + let key_range = sst.key_range.unwrap(); + let sub_level_id = if level.level_idx > 0 { + None + } else { + Some(ScalarImpl::Int64(level.sub_level_id as _)) + }; + sstables.push(OwnedRow::new(vec![ + Some(ScalarImpl::Int64(sst.sst_id as _)), + Some(ScalarImpl::Int64(sst.object_id as _)), + Some(ScalarImpl::Int64(cg.group_id as _)), + Some(ScalarImpl::Int32(level.level_idx as _)), + sub_level_id, + Some(ScalarImpl::Int32(level.level_type as _)), + Some(ScalarImpl::Bytea(key_range.left.into())), + Some(ScalarImpl::Bytea(key_range.right.into())), + Some(ScalarImpl::Bool(key_range.right_exclusive)), + Some(ScalarImpl::Int64(sst.file_size as _)), + Some(ScalarImpl::Int64(sst.meta_offset as _)), + Some(ScalarImpl::Int64(sst.stale_key_count as _)), + Some(ScalarImpl::Int64(sst.total_key_count as _)), + Some(ScalarImpl::Int64(sst.min_epoch as _)), + Some(ScalarImpl::Int64(sst.max_epoch as _)), + Some(ScalarImpl::Int64(sst.uncompressed_file_size as _)), + Some(ScalarImpl::Int64(sst.range_tombstone_count as _)), + Some(ScalarImpl::Int32(sst.bloom_filter_kind as _)), + Some(ScalarImpl::Jsonb(json!(sst.table_ids).into())), + ])); + } + } + } + sstables +} From fd7e485a9af902c3b18f49e76e87445bdd79f447 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 26 Sep 2023 14:58:04 +0800 Subject: [PATCH 15/17] chore: bump risingwave version in docker compose to 1.2.0 (#12530) Signed-off-by: Bugen Zhao --- docker/docker-compose.yml | 8 ++++---- docs/memory-profiling.md | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index c7f2db4cb737d..328ebe9b7bacb 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -39,7 +39,7 @@ services: timeout: 5s retries: 5 compute-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.0.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" command: - compute-node - "--listen-addr" @@ -126,7 +126,7 @@ services: timeout: 5s retries: 5 frontend-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.0.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" command: - frontend-node - "--listen-addr" @@ -185,7 +185,7 @@ services: timeout: 5s retries: 5 meta-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.0.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" command: - meta-node - "--listen-addr" @@ -301,7 +301,7 @@ services: timeout: 5s retries: 5 connector-node: - image: ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.0.0} + image: ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0} entrypoint: "/risingwave/bin/connector-node/start-service.sh" ports: - 50051 diff --git a/docs/memory-profiling.md b/docs/memory-profiling.md index fec324770f0c8..759807ba1e9ed 100644 --- a/docs/memory-profiling.md +++ b/docs/memory-profiling.md @@ -158,7 +158,7 @@ cp ./target/release/examples/addr2line Find a Linux machine and use `docker` command to start an environment with the specific RisingWave version. Here, `-v $(pwd):/dumps` mounts current directory to `/dumps` folder inside the container, so that you don't need to copy the files in and out. ```bash -docker run -it --rm --entrypoint /bin/bash -v $(pwd):/dumps ghcr.io/risingwavelabs/risingwave:v1.0.0 +docker run -it --rm --entrypoint /bin/bash -v $(pwd):/dumps ghcr.io/risingwavelabs/risingwave:latest ``` From 7e700e700f19891fd588082e612e8c58d55c41ea Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Sep 2023 15:04:02 +0800 Subject: [PATCH 16/17] refactor(sink): decouple sink formatter and payload writer (#12515) --- src/connector/src/sink/doris.rs | 26 +-- src/connector/src/sink/encoder/json.rs | 23 +-- .../src/sink/formatter/debezium_json.rs | 51 +++--- src/connector/src/sink/formatter/mod.rs | 71 +++++++- src/connector/src/sink/kafka.rs | 166 +++++------------- src/connector/src/sink/kinesis.rs | 96 +++------- src/connector/src/sink/mod.rs | 2 +- src/connector/src/sink/nats.rs | 5 +- src/connector/src/sink/pulsar.rs | 85 ++++----- src/connector/src/sink/remote.rs | 7 +- src/connector/src/sink/utils.rs | 30 +--- 11 files changed, 248 insertions(+), 314 deletions(-) diff --git a/src/connector/src/sink/doris.rs b/src/connector/src/sink/doris.rs index c19365fcc51c4..82e6454663f48 100644 --- a/src/connector/src/sink/doris.rs +++ b/src/connector/src/sink/doris.rs @@ -26,9 +26,9 @@ use serde_json::Value; use serde_with::serde_as; use super::doris_connector::{DorisField, DorisInsert, DorisInsertClient, DORIS_DELETE_SIGN}; -use super::utils::doris_rows_to_json; use super::{SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use crate::common::DorisCommon; +use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; pub const DORIS_SINK: &str = "doris"; @@ -192,7 +192,7 @@ pub struct DorisSinkWriter { client: DorisInsertClient, is_append_only: bool, insert: Option, - decimal_map: HashMap, + row_encoder: JsonEncoder, } impl DorisSinkWriter { @@ -232,12 +232,17 @@ impl DorisSinkWriter { let insert = Some(doris_insert_client.build().await?); Ok(Self { config, - schema, + schema: schema.clone(), pk_indices, client: doris_insert_client, is_append_only, insert, - decimal_map, + row_encoder: JsonEncoder::new_with_doris( + schema, + None, + TimestampHandlingMode::String, + decimal_map, + ), }) } @@ -246,9 +251,7 @@ impl DorisSinkWriter { if op != Op::Insert { continue; } - let row_json_string = - Value::Object(doris_rows_to_json(row, &self.schema, &self.decimal_map)?) - .to_string(); + let row_json_string = Value::Object(self.row_encoder.encode(row)?).to_string(); self.insert .as_mut() .ok_or_else(|| SinkError::Doris("Can't find doris sink insert".to_string()))? @@ -262,8 +265,7 @@ impl DorisSinkWriter { for (op, row) in chunk.rows() { match op { Op::Insert => { - let mut row_json_value = - doris_rows_to_json(row, &self.schema, &self.decimal_map)?; + let mut row_json_value = self.row_encoder.encode(row)?; row_json_value.insert( DORIS_DELETE_SIGN.to_string(), Value::String("0".to_string()), @@ -279,8 +281,7 @@ impl DorisSinkWriter { .await?; } Op::Delete => { - let mut row_json_value = - doris_rows_to_json(row, &self.schema, &self.decimal_map)?; + let mut row_json_value = self.row_encoder.encode(row)?; row_json_value.insert( DORIS_DELETE_SIGN.to_string(), Value::String("1".to_string()), @@ -297,8 +298,7 @@ impl DorisSinkWriter { } Op::UpdateDelete => {} Op::UpdateInsert => { - let mut row_json_value = - doris_rows_to_json(row, &self.schema, &self.decimal_map)?; + let mut row_json_value = self.row_encoder.encode(row)?; row_json_value.insert( DORIS_DELETE_SIGN.to_string(), Value::String("0".to_string()), diff --git a/src/connector/src/sink/encoder/json.rs b/src/connector/src/sink/encoder/json.rs index 9264df0d12a97..6add09b2cb86e 100644 --- a/src/connector/src/sink/encoder/json.rs +++ b/src/connector/src/sink/encoder/json.rs @@ -27,17 +27,18 @@ use serde_json::{json, Map, Value}; use super::{CustomJsonType, Result, RowEncoder, SerTo, TimestampHandlingMode}; use crate::sink::SinkError; -pub struct JsonEncoder<'a> { - schema: &'a Schema, - col_indices: Option<&'a [usize]>, + +pub struct JsonEncoder { + schema: Schema, + col_indices: Option>, timestamp_handling_mode: TimestampHandlingMode, custom_json_type: CustomJsonType, } -impl<'a> JsonEncoder<'a> { +impl JsonEncoder { pub fn new( - schema: &'a Schema, - col_indices: Option<&'a [usize]>, + schema: Schema, + col_indices: Option>, timestamp_handling_mode: TimestampHandlingMode, ) -> Self { Self { @@ -49,8 +50,8 @@ impl<'a> JsonEncoder<'a> { } pub fn new_with_doris( - schema: &'a Schema, - col_indices: Option<&'a [usize]>, + schema: Schema, + col_indices: Option>, timestamp_handling_mode: TimestampHandlingMode, map: HashMap, ) -> Self { @@ -63,15 +64,15 @@ impl<'a> JsonEncoder<'a> { } } -impl<'a> RowEncoder for JsonEncoder<'a> { +impl RowEncoder for JsonEncoder { type Output = Map; fn schema(&self) -> &Schema { - self.schema + &self.schema } fn col_indices(&self) -> Option<&[usize]> { - self.col_indices + self.col_indices.as_ref().map(Vec::as_ref) } fn encode_cols( diff --git a/src/connector/src/sink/formatter/debezium_json.rs b/src/connector/src/sink/formatter/debezium_json.rs index f5b48e836c64d..f002d17e75dfe 100644 --- a/src/connector/src/sink/formatter/debezium_json.rs +++ b/src/connector/src/sink/formatter/debezium_json.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::time::{SystemTime, UNIX_EPOCH}; + use risingwave_common::array::Op; use risingwave_common::catalog::{Field, Schema}; use serde_json::{json, Map, Value}; @@ -39,36 +41,43 @@ fn concat_debezium_name_field(db_name: &str, sink_from_name: &str, value: &str) DEBEZIUM_NAME_FIELD_PREFIX.to_owned() + "." + db_name + "." + sink_from_name + "." + value } -pub struct DebeziumJsonFormatter<'a> { - schema: &'a Schema, - pk_indices: &'a [usize], - db_name: &'a str, - sink_from_name: &'a str, +pub struct DebeziumJsonFormatter { + schema: Schema, + pk_indices: Vec, + db_name: String, + sink_from_name: String, opts: DebeziumAdapterOpts, - ts_ms: u64, + key_encoder: JsonEncoder, + val_encoder: JsonEncoder, } -impl<'a> DebeziumJsonFormatter<'a> { +impl DebeziumJsonFormatter { pub fn new( - schema: &'a Schema, - pk_indices: &'a [usize], - db_name: &'a str, - sink_from_name: &'a str, + schema: Schema, + pk_indices: Vec, + db_name: String, + sink_from_name: String, opts: DebeziumAdapterOpts, - ts_ms: u64, ) -> Self { + let key_encoder = JsonEncoder::new( + schema.clone(), + Some(pk_indices.clone()), + TimestampHandlingMode::Milli, + ); + let val_encoder = JsonEncoder::new(schema.clone(), None, TimestampHandlingMode::Milli); Self { schema, pk_indices, db_name, sink_from_name, opts, - ts_ms, + key_encoder, + val_encoder, } } } -impl<'a> SinkFormatter for DebeziumJsonFormatter<'a> { +impl SinkFormatter for DebeziumJsonFormatter { type K = Value; type V = Value; @@ -83,8 +92,13 @@ impl<'a> SinkFormatter for DebeziumJsonFormatter<'a> { db_name, sink_from_name, opts, - ts_ms, + key_encoder, + val_encoder, } = self; + let ts_ms = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() as u64; let source_field = json!({ "db": db_name, "table": sink_from_name, @@ -92,10 +106,6 @@ impl<'a> SinkFormatter for DebeziumJsonFormatter<'a> { let mut update_cache: Option> = None; - let key_encoder = - JsonEncoder::new(schema, Some(pk_indices), TimestampHandlingMode::Milli); - let val_encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); - for (op, row) in chunk.rows() { let event_key_object: Option = Some(json!({ "schema": json!({ @@ -338,7 +348,8 @@ mod tests { }, ]); - let json_chunk = chunk_to_json(chunk, &schema).unwrap(); + let encoder = JsonEncoder::new(schema.clone(), None, TimestampHandlingMode::Milli); + let json_chunk = chunk_to_json(chunk, &encoder).unwrap(); let schema_json = schema_to_json(&schema, "test_db", "test_table"); assert_eq!(schema_json, serde_json::from_str::("{\"fields\":[{\"field\":\"before\",\"fields\":[{\"field\":\"v1\",\"optional\":true,\"type\":\"int32\"},{\"field\":\"v2\",\"optional\":true,\"type\":\"float\"},{\"field\":\"v3\",\"optional\":true,\"type\":\"string\"}],\"name\":\"RisingWave.test_db.test_table.Key\",\"optional\":true,\"type\":\"struct\"},{\"field\":\"after\",\"fields\":[{\"field\":\"v1\",\"optional\":true,\"type\":\"int32\"},{\"field\":\"v2\",\"optional\":true,\"type\":\"float\"},{\"field\":\"v3\",\"optional\":true,\"type\":\"string\"}],\"name\":\"RisingWave.test_db.test_table.Key\",\"optional\":true,\"type\":\"struct\"},{\"field\":\"source\",\"fields\":[{\"field\":\"db\",\"optional\":false,\"type\":\"string\"},{\"field\":\"table\",\"optional\":true,\"type\":\"string\"}],\"name\":\"RisingWave.test_db.test_table.Source\",\"optional\":false,\"type\":\"struct\"},{\"field\":\"op\",\"optional\":false,\"type\":\"string\"},{\"field\":\"ts_ms\",\"optional\":false,\"type\":\"int64\"}],\"name\":\"RisingWave.test_db.test_table.Envelope\",\"optional\":false,\"type\":\"struct\"}").unwrap()); assert_eq!( diff --git a/src/connector/src/sink/formatter/mod.rs b/src/connector/src/sink/formatter/mod.rs index 585752b03327c..89e8369d93819 100644 --- a/src/connector/src/sink/formatter/mod.rs +++ b/src/connector/src/sink/formatter/mod.rs @@ -12,9 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::anyhow; use risingwave_common::array::StreamChunk; -use crate::sink::Result; +use crate::sink::{Result, SinkError, SINK_TYPE_DEBEZIUM, SINK_TYPE_UPSERT}; mod append_only; mod debezium_json; @@ -22,8 +23,11 @@ mod upsert; pub use append_only::AppendOnlyFormatter; pub use debezium_json::{DebeziumAdapterOpts, DebeziumJsonFormatter}; +use risingwave_common::catalog::Schema; pub use upsert::UpsertFormatter; +use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; + /// Transforms a `StreamChunk` into a sequence of key-value pairs according a specific format, /// for example append-only, upsert or debezium. pub trait SinkFormatter { @@ -50,3 +54,68 @@ macro_rules! tri { } }; } + +#[expect(clippy::enum_variant_names)] +pub enum SinkFormatterImpl { + AppendOnlyJson(AppendOnlyFormatter), + UpsertJson(UpsertFormatter), + DebeziumJson(DebeziumJsonFormatter), +} + +impl SinkFormatterImpl { + pub fn new( + formatter_type: &str, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + db_name: String, + sink_from_name: String, + ) -> Result { + if is_append_only { + let key_encoder = JsonEncoder::new( + schema.clone(), + Some(pk_indices), + TimestampHandlingMode::Milli, + ); + let val_encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); + + let formatter = AppendOnlyFormatter::new(key_encoder, val_encoder); + Ok(SinkFormatterImpl::AppendOnlyJson(formatter)) + } else if formatter_type == SINK_TYPE_DEBEZIUM { + Ok(SinkFormatterImpl::DebeziumJson(DebeziumJsonFormatter::new( + schema, + pk_indices, + db_name, + sink_from_name, + DebeziumAdapterOpts::default(), + ))) + } else if formatter_type == SINK_TYPE_UPSERT { + let key_encoder = JsonEncoder::new( + schema.clone(), + Some(pk_indices), + TimestampHandlingMode::Milli, + ); + let val_encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); + + // Initialize the upsert_stream + let formatter = UpsertFormatter::new(key_encoder, val_encoder); + Ok(SinkFormatterImpl::UpsertJson(formatter)) + } else { + Err(SinkError::Config(anyhow!( + "unsupported upsert sink type {}", + formatter_type + ))) + } + } +} + +#[macro_export] +macro_rules! dispatch_sink_formatter_impl { + ($impl:expr, $name:ident, $body:expr) => { + match $impl { + SinkFormatterImpl::AppendOnlyJson($name) => $body, + SinkFormatterImpl::UpsertJson($name) => $body, + SinkFormatterImpl::DebeziumJson($name) => $body, + } + }; +} diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index 4ae886d15ce0c..85894792cd7ee 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -15,7 +15,7 @@ use std::collections::{HashMap, VecDeque}; use std::fmt::Debug; use std::sync::Arc; -use std::time::{Duration, SystemTime, UNIX_EPOCH}; +use std::time::Duration; use anyhow::anyhow; use futures::future::try_join_all; @@ -31,15 +31,12 @@ use serde_derive::{Deserialize, Serialize}; use serde_with::{serde_as, DisplayFromStr}; use strum_macros::{Display, EnumString}; -use super::encoder::{JsonEncoder, TimestampHandlingMode}; -use super::formatter::{ - AppendOnlyFormatter, DebeziumAdapterOpts, DebeziumJsonFormatter, UpsertFormatter, -}; use super::{ FormattedSink, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_DEBEZIUM, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; use crate::common::KafkaCommon; +use crate::sink::formatter::SinkFormatterImpl; use crate::sink::{ DummySinkCommitCoordinator, Result, SinkWriterParam, SinkWriterV1, SinkWriterV1Adapter, }; @@ -47,6 +44,7 @@ use crate::source::kafka::{KafkaProperties, KafkaSplitEnumerator, PrivateLinkPro use crate::source::{SourceEnumeratorContext, SplitEnumerator}; use crate::{ deserialize_bool_from_string, deserialize_duration_from_string, deserialize_u32_from_string, + dispatch_sink_formatter_impl, }; pub const KAFKA_SINK: &str = "kafka"; @@ -301,16 +299,18 @@ impl Sink for KafkaSink { type Coordinator = DummySinkCommitCoordinator; type Writer = SinkWriterV1Adapter; - async fn new_writer(&self, writer_param: SinkWriterParam) -> Result { + async fn new_writer(&self, _writer_param: SinkWriterParam) -> Result { Ok(SinkWriterV1Adapter::new( KafkaSinkWriter::new( self.config.clone(), - self.schema.clone(), - self.pk_indices.clone(), - self.is_append_only, - self.db_name.clone(), - self.sink_from_name.clone(), - format!("sink-{:?}", writer_param.executor_id), + SinkFormatterImpl::new( + &self.config.r#type, + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + self.db_name.clone(), + self.sink_from_name.clone(), + )?, ) .await?, )) @@ -338,41 +338,25 @@ impl Sink for KafkaSink { } } -#[derive(Debug, Clone, PartialEq, enum_as_inner::EnumAsInner)] -enum KafkaSinkState { - Init, - // State running with epoch. - Running(u64), -} - /// The delivery buffer queue size /// When the `DeliveryFuture` the current `future_delivery_buffer` /// is buffering is greater than this size, then enforcing commit once const KAFKA_WRITER_MAX_QUEUE_SIZE: usize = 65536; +struct KafkaPayloadWriter { + inner: FutureProducer, + future_delivery_buffer: VecDeque, + config: KafkaConfig, +} + pub struct KafkaSinkWriter { pub config: KafkaConfig, - pub inner: FutureProducer, - identifier: String, - state: KafkaSinkState, - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - future_delivery_buffer: VecDeque, - db_name: String, - sink_from_name: String, + payload_writer: KafkaPayloadWriter, + formatter: SinkFormatterImpl, } impl KafkaSinkWriter { - pub async fn new( - mut config: KafkaConfig, - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - db_name: String, - sink_from_name: String, - identifier: String, - ) -> Result { + pub async fn new(mut config: KafkaConfig, formatter: SinkFormatterImpl) -> Result { let inner: FutureProducer = { let mut c = ClientConfig::new(); @@ -400,18 +384,17 @@ impl KafkaSinkWriter { Ok(KafkaSinkWriter { config: config.clone(), - inner, - identifier, - state: KafkaSinkState::Init, - schema, - pk_indices, - is_append_only, - future_delivery_buffer: VecDeque::new(), - db_name, - sink_from_name, + payload_writer: KafkaPayloadWriter { + inner, + future_delivery_buffer: VecDeque::new(), + config, + }, + formatter, }) } +} +impl KafkaPayloadWriter { /// The actual `send_result` function, will be called when the `KafkaSinkWriter` needs to sink /// messages async fn send_result<'a, K, P>( @@ -537,57 +520,9 @@ impl KafkaSinkWriter { Ok(()) } - - async fn debezium_update(&mut self, chunk: StreamChunk, ts_ms: u64) -> Result<()> { - // TODO: Remove the clones here, only to satisfy borrow checker at present - let schema = self.schema.clone(); - let pk_indices = self.pk_indices.clone(); - let db_name = self.db_name.clone(); - let sink_from_name = self.sink_from_name.clone(); - - // Initialize the dbz_stream - let f = DebeziumJsonFormatter::new( - &schema, - &pk_indices, - &db_name, - &sink_from_name, - DebeziumAdapterOpts::default(), - ts_ms, - ); - - self.write_chunk(chunk, f).await - } - - async fn upsert(&mut self, chunk: StreamChunk) -> Result<()> { - // TODO: Remove the clones here, only to satisfy borrow checker at present - let schema = self.schema.clone(); - let pk_indices = self.pk_indices.clone(); - let key_encoder = - JsonEncoder::new(&schema, Some(&pk_indices), TimestampHandlingMode::Milli); - let val_encoder = JsonEncoder::new(&schema, None, TimestampHandlingMode::Milli); - - // Initialize the upsert_stream - let f = UpsertFormatter::new(key_encoder, val_encoder); - - self.write_chunk(chunk, f).await - } - - async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { - // TODO: Remove the clones here, only to satisfy borrow checker at present - let schema = self.schema.clone(); - let pk_indices = self.pk_indices.clone(); - let key_encoder = - JsonEncoder::new(&schema, Some(&pk_indices), TimestampHandlingMode::Milli); - let val_encoder = JsonEncoder::new(&schema, None, TimestampHandlingMode::Milli); - - // Initialize the append_only_stream - let f = AppendOnlyFormatter::new(key_encoder, val_encoder); - - self.write_chunk(chunk, f).await - } } -impl FormattedSink for KafkaSinkWriter { +impl FormattedSink for KafkaPayloadWriter { type K = Vec; type V = Vec; @@ -599,25 +534,9 @@ impl FormattedSink for KafkaSinkWriter { #[async_trait::async_trait] impl SinkWriterV1 for KafkaSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - if self.is_append_only { - // Append-only - self.append_only(chunk).await - } else { - // Debezium - if self.config.r#type == SINK_TYPE_DEBEZIUM { - self.debezium_update( - chunk, - SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_millis() as u64, - ) - .await - } else { - // Upsert - self.upsert(chunk).await - } - } + dispatch_sink_formatter_impl!(&self.formatter, formatter, { + self.payload_writer.write_chunk(chunk, formatter).await + }) } /// --------------------------------------------------------------------------------------- @@ -632,7 +551,7 @@ impl SinkWriterV1 for KafkaSinkWriter { async fn commit(&mut self) -> Result<()> { // Group delivery (await the `FutureRecord`) here - self.commit_inner().await?; + self.payload_writer.commit_inner().await?; Ok(()) } @@ -648,6 +567,8 @@ mod test { use risingwave_common::types::DataType; use super::*; + use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; + use crate::sink::formatter::AppendOnlyFormatter; #[test] fn parse_rdkafka_props() { @@ -822,12 +743,14 @@ mod test { // Create the actual sink writer to Kafka let mut sink = KafkaSinkWriter::new( kafka_config.clone(), - schema, - pk_indices, - true, - "test_sink_1".to_string(), - "test_db".into(), - "test_table".into(), + SinkFormatterImpl::AppendOnlyJson(AppendOnlyFormatter::new( + JsonEncoder::new( + schema.clone(), + Some(pk_indices), + TimestampHandlingMode::Milli, + ), + JsonEncoder::new(schema, None, TimestampHandlingMode::Milli), + )), ) .await .unwrap(); @@ -838,6 +761,7 @@ mod test { println!("epoch: {}", i); for j in 0..100 { match sink + .payload_writer .send_result( FutureRecord::to(kafka_config.common.topic.as_str()) .payload(format!("value-{}", j).as_bytes()) diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index 7c43abcbe82a7..114070ca1e440 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::collections::HashMap; -use std::time::{SystemTime, UNIX_EPOCH}; use anyhow::anyhow; use aws_sdk_kinesis::error::DisplayErrorContext; @@ -27,12 +26,10 @@ use serde_with::serde_as; use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tokio_retry::Retry; -use super::formatter::{ - AppendOnlyFormatter, DebeziumAdapterOpts, DebeziumJsonFormatter, UpsertFormatter, -}; use super::{FormattedSink, SinkParam}; use crate::common::KinesisCommon; -use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; +use crate::dispatch_sink_formatter_impl; +use crate::sink::formatter::SinkFormatterImpl; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkError, SinkWriter, SinkWriterParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_DEBEZIUM, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, @@ -134,15 +131,15 @@ impl KinesisSinkConfig { } } -#[derive(Debug)] pub struct KinesisSinkWriter { pub config: KinesisSinkConfig, - schema: Schema, - pk_indices: Vec, + formatter: SinkFormatterImpl, + payload_writer: KinesisSinkPayloadWriter, +} + +struct KinesisSinkPayloadWriter { client: KinesisClient, - is_append_only: bool, - db_name: String, - sink_from_name: String, + config: KinesisSinkConfig, } impl KinesisSinkWriter { @@ -154,22 +151,27 @@ impl KinesisSinkWriter { db_name: String, sink_from_name: String, ) -> Result { + let formatter = SinkFormatterImpl::new( + &config.r#type, + schema, + pk_indices, + is_append_only, + db_name, + sink_from_name, + )?; let client = config .common .build_client() .await .map_err(SinkError::Kinesis)?; Ok(Self { - config, - schema, - pk_indices, - client, - is_append_only, - db_name, - sink_from_name, + config: config.clone(), + formatter, + payload_writer: KinesisSinkPayloadWriter { client, config }, }) } - +} +impl KinesisSinkPayloadWriter { async fn put_record(&self, key: &str, payload: Vec) -> Result { let payload = Blob::new(payload); // todo: switch to put_records() for batching @@ -199,44 +201,9 @@ impl KinesisSinkWriter { )) }) } - - async fn upsert(mut self: &Self, chunk: StreamChunk) -> Result<()> { - let key_encoder = JsonEncoder::new( - &self.schema, - Some(&self.pk_indices), - TimestampHandlingMode::Milli, - ); - let val_encoder = JsonEncoder::new(&self.schema, None, TimestampHandlingMode::Milli); - let f = UpsertFormatter::new(key_encoder, val_encoder); - self.write_chunk(chunk, f).await - } - - async fn append_only(mut self: &Self, chunk: StreamChunk) -> Result<()> { - let key_encoder = JsonEncoder::new( - &self.schema, - Some(&self.pk_indices), - TimestampHandlingMode::Milli, - ); - let val_encoder = JsonEncoder::new(&self.schema, None, TimestampHandlingMode::Milli); - let f = AppendOnlyFormatter::new(key_encoder, val_encoder); - self.write_chunk(chunk, f).await - } - - async fn debezium_update(mut self: &Self, chunk: StreamChunk, ts_ms: u64) -> Result<()> { - let f = DebeziumJsonFormatter::new( - &self.schema, - &self.pk_indices, - &self.db_name, - &self.sink_from_name, - DebeziumAdapterOpts::default(), - ts_ms, - ); - - self.write_chunk(chunk, f).await - } } -impl FormattedSink for &KinesisSinkWriter { +impl FormattedSink for KinesisSinkPayloadWriter { type K = String; type V = Vec; @@ -250,22 +217,9 @@ impl FormattedSink for &KinesisSinkWriter { #[async_trait::async_trait] impl SinkWriter for KinesisSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - if self.is_append_only { - self.append_only(chunk).await - } else if self.config.r#type == SINK_TYPE_DEBEZIUM { - self.debezium_update( - chunk, - SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_millis() as u64, - ) - .await - } else if self.config.r#type == SINK_TYPE_UPSERT { - self.upsert(chunk).await - } else { - unreachable!() - } + dispatch_sink_formatter_impl!(&self.formatter, formatter, { + self.payload_writer.write_chunk(chunk, formatter).await + }) } async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index f1f12ed323d7b..449f4f541f0e4 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -230,7 +230,7 @@ pub trait FormattedSink { async fn write_chunk( &mut self, chunk: StreamChunk, - formatter: F, + formatter: &F, ) -> Result<()> where F::K: SerTo, diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index e04235b6f586e..57b9e572c0615 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -27,6 +27,7 @@ use tokio_retry::Retry; use super::utils::chunk_to_json; use super::{DummySinkCommitCoordinator, SinkWriter, SinkWriterParam}; use crate::common::NatsCommon; +use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; use crate::sink::{Result, Sink, SinkError, SINK_TYPE_APPEND_ONLY}; pub const NATS_SINK: &str = "nats"; @@ -52,6 +53,7 @@ pub struct NatsSinkWriter { pub config: NatsConfig, context: Context, schema: Schema, + json_encoder: JsonEncoder, } /// Basic data types for use with the nats interface @@ -118,6 +120,7 @@ impl NatsSinkWriter { config: config.clone(), context, schema: schema.clone(), + json_encoder: JsonEncoder::new(schema, None, TimestampHandlingMode::Milli), }) } @@ -125,7 +128,7 @@ impl NatsSinkWriter { Retry::spawn( ExponentialBackoff::from_millis(100).map(jitter).take(3), || async { - let data = chunk_to_json(chunk.clone(), &self.schema).unwrap(); + let data = chunk_to_json(chunk.clone(), &self.json_encoder).unwrap(); for item in data { self.context .publish(self.config.common.subject.clone(), item.into()) diff --git a/src/connector/src/sink/pulsar.rs b/src/connector/src/sink/pulsar.rs index d1cbc0b5a527c..8dc478f7dd82d 100644 --- a/src/connector/src/sink/pulsar.rs +++ b/src/connector/src/sink/pulsar.rs @@ -27,15 +27,14 @@ use risingwave_common::catalog::Schema; use serde::Deserialize; use serde_with::{serde_as, DisplayFromStr}; -use super::encoder::{JsonEncoder, TimestampHandlingMode}; -use super::formatter::{AppendOnlyFormatter, UpsertFormatter}; use super::{ FormattedSink, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; use crate::common::PulsarCommon; -use crate::deserialize_duration_from_string; +use crate::sink::formatter::SinkFormatterImpl; use crate::sink::{DummySinkCommitCoordinator, Result}; +use crate::{deserialize_duration_from_string, dispatch_sink_formatter_impl}; pub const PULSAR_SINK: &str = "pulsar"; @@ -142,6 +141,8 @@ pub struct PulsarSink { schema: Schema, downstream_pk: Vec, is_append_only: bool, + db_name: String, + sink_from_name: String, } impl PulsarSink { @@ -151,6 +152,8 @@ impl PulsarSink { schema: param.schema(), downstream_pk: param.downstream_pk, is_append_only: param.sink_type.is_append_only(), + db_name: param.db_name, + sink_from_name: param.sink_from_name, } } } @@ -166,6 +169,8 @@ impl Sink for PulsarSink { self.schema.clone(), self.downstream_pk.clone(), self.is_append_only, + self.db_name.clone(), + self.sink_from_name.clone(), ) .await } @@ -188,12 +193,14 @@ impl Sink for PulsarSink { } pub struct PulsarSinkWriter { + payload_writer: PulsarPayloadWriter, + formatter: SinkFormatterImpl, +} + +struct PulsarPayloadWriter { pulsar: Pulsar, producer: Producer, config: PulsarConfig, - schema: Schema, - downstream_pk: Vec, - is_append_only: bool, send_future_buffer: VecDeque, } @@ -203,20 +210,32 @@ impl PulsarSinkWriter { schema: Schema, downstream_pk: Vec, is_append_only: bool, + db_name: String, + sink_from_name: String, ) -> Result { - let pulsar = config.common.build_client().await?; - let producer = build_pulsar_producer(&pulsar, &config).await?; - Ok(Self { - pulsar, - producer, - config, + let formatter = SinkFormatterImpl::new( + &config.r#type, schema, downstream_pk, is_append_only, - send_future_buffer: VecDeque::new(), + db_name, + sink_from_name, + )?; + let pulsar = config.common.build_client().await?; + let producer = build_pulsar_producer(&pulsar, &config).await?; + Ok(Self { + formatter, + payload_writer: PulsarPayloadWriter { + pulsar, + producer, + config, + send_future_buffer: VecDeque::new(), + }, }) } +} +impl PulsarPayloadWriter { async fn send_message(&mut self, message: Message) -> Result<()> { let mut success_flag = false; let mut connection_err = None; @@ -276,34 +295,6 @@ impl PulsarSinkWriter { Ok(()) } - async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { - // TODO: Remove the clones here, only to satisfy borrow checker at present - let schema = self.schema.clone(); - let downstream_pk = self.downstream_pk.clone(); - let key_encoder = - JsonEncoder::new(&schema, Some(&downstream_pk), TimestampHandlingMode::Milli); - let val_encoder = JsonEncoder::new(&schema, None, TimestampHandlingMode::Milli); - - // Initialize the append_only_stream - let f = AppendOnlyFormatter::new(key_encoder, val_encoder); - - self.write_chunk(chunk, f).await - } - - async fn upsert(&mut self, chunk: StreamChunk) -> Result<()> { - // TODO: Remove the clones here, only to satisfy borrow checker at present - let schema = self.schema.clone(); - let downstream_pk = self.downstream_pk.clone(); - let key_encoder = - JsonEncoder::new(&schema, Some(&downstream_pk), TimestampHandlingMode::Milli); - let val_encoder = JsonEncoder::new(&schema, None, TimestampHandlingMode::Milli); - - // Initialize the upsert_stream - let f = UpsertFormatter::new(key_encoder, val_encoder); - - self.write_chunk(chunk, f).await - } - async fn commit_inner(&mut self) -> Result<()> { self.producer .send_batch() @@ -320,7 +311,7 @@ impl PulsarSinkWriter { } } -impl FormattedSink for PulsarSinkWriter { +impl FormattedSink for PulsarPayloadWriter { type K = String; type V = Vec; @@ -332,11 +323,9 @@ impl FormattedSink for PulsarSinkWriter { #[async_trait] impl SinkWriter for PulsarSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - if self.is_append_only { - self.append_only(chunk).await - } else { - self.upsert(chunk).await - } + dispatch_sink_formatter_impl!(&self.formatter, formatter, { + self.payload_writer.write_chunk(chunk, formatter).await + }) } async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { @@ -345,7 +334,7 @@ impl SinkWriter for PulsarSinkWriter { async fn barrier(&mut self, is_checkpoint: bool) -> Result { if is_checkpoint { - self.commit_inner().await?; + self.payload_writer.commit_inner().await?; } Ok(()) diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index 004f8ac46d128..f6fa615e44c53 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -245,7 +245,6 @@ impl Sink for CoordinatedRemoteSink { pub type RemoteSinkWriter = RemoteSinkWriterInner<()>; pub type CoordinatedRemoteSinkWriter = RemoteSinkWriterInner>; -#[derive(Debug)] pub struct RemoteSinkWriterInner { pub connector_type: String, properties: HashMap, @@ -254,6 +253,7 @@ pub struct RemoteSinkWriterInner { schema: Schema, payload_format: SinkPayloadFormat, stream_handle: SinkWriterStreamHandle, + json_encoder: JsonEncoder, _phantom: PhantomData, } @@ -291,6 +291,7 @@ impl RemoteSinkWriterInner { schema: param.schema(), stream_handle, payload_format: connector_params.sink_payload_format, + json_encoder: JsonEncoder::new(param.schema(), None, TimestampHandlingMode::String), _phantom: PhantomData, }) } @@ -331,6 +332,7 @@ impl RemoteSinkWriterInner { properties, epoch: None, batch_id: 0, + json_encoder: JsonEncoder::new(schema.clone(), None, TimestampHandlingMode::String), schema, stream_handle, payload_format: SinkPayloadFormat::Json, @@ -387,9 +389,8 @@ where let payload = match self.payload_format { SinkPayloadFormat::Json => { let mut row_ops = Vec::with_capacity(chunk.cardinality()); - let enc = JsonEncoder::new(&self.schema, None, TimestampHandlingMode::String); for (op, row_ref) in chunk.rows() { - let map = enc.encode(row_ref)?; + let map = self.json_encoder.encode(row_ref)?; let row_op = RowOp { op_type: op.to_protobuf() as i32, line: serde_json::to_string(&map) diff --git a/src/connector/src/sink/utils.rs b/src/connector/src/sink/utils.rs index da9e92b6ae0aa..967c3fc43ba30 100644 --- a/src/connector/src/sink/utils.rs +++ b/src/connector/src/sink/utils.rs @@ -1,5 +1,3 @@ -use std::collections::HashMap; - // Copyright 2023 RisingWave Labs // // Licensed under the Apache License, Version 2.0 (the "License"); @@ -13,16 +11,15 @@ use std::collections::HashMap; // 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::array::{RowRef, StreamChunk}; -use risingwave_common::catalog::Schema; -use serde_json::{Map, Value}; -use super::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use risingwave_common::array::StreamChunk; +use serde_json::Value; + +use super::encoder::{JsonEncoder, RowEncoder}; use crate::sink::Result; -pub fn chunk_to_json(chunk: StreamChunk, schema: &Schema) -> Result> { - let encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); - let mut records: Vec = Vec::with_capacity(chunk.cardinality()); +pub fn chunk_to_json(chunk: StreamChunk, encoder: &JsonEncoder) -> Result> { + let mut records: Vec = Vec::with_capacity(chunk.capacity()); for (_, row) in chunk.rows() { let record = Value::Object(encoder.encode(row)?); records.push(record.to_string()); @@ -30,18 +27,3 @@ pub fn chunk_to_json(chunk: StreamChunk, schema: &Schema) -> Result> Ok(records) } - -pub fn doris_rows_to_json( - row: RowRef<'_>, - schema: &Schema, - decimal_map: &HashMap, -) -> Result> { - let encoder = JsonEncoder::new_with_doris( - schema, - None, - TimestampHandlingMode::String, - decimal_map.clone(), - ); - let map = encoder.encode(row)?; - Ok(map) -} From 5eeef12e02e5d245a7ff37a7492bdfee2e90e1e7 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Sep 2023 15:07:59 +0800 Subject: [PATCH 17/17] chore(java-binding): refine stream chunk benchmark (#12504) --- .../java/binding/ArrayListBenchmark.java | 15 ++------ .../java/binding/StreamchunkBenchmark.java | 36 ++++++++++++------- src/java_binding/make-java-binding.toml | 9 ++--- 3 files changed, 29 insertions(+), 31 deletions(-) diff --git a/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/ArrayListBenchmark.java b/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/ArrayListBenchmark.java index 6540033371d34..c05cf23d2c582 100644 --- a/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/ArrayListBenchmark.java +++ b/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/ArrayListBenchmark.java @@ -20,8 +20,8 @@ import java.util.concurrent.TimeUnit; import org.openjdk.jmh.annotations.*; -@Warmup(iterations = 10, time = 1, timeUnit = TimeUnit.MILLISECONDS) -@Measurement(iterations = 20, time = 1, timeUnit = TimeUnit.MILLISECONDS) +@Warmup(iterations = 2, time = 1, timeUnit = TimeUnit.MILLISECONDS, batchSize = 10) +@Measurement(iterations = 5, time = 1, timeUnit = TimeUnit.MILLISECONDS, batchSize = 10) @Fork(value = 1) @BenchmarkMode(org.openjdk.jmh.annotations.Mode.AverageTime) @OutputTimeUnit(TimeUnit.MICROSECONDS) @@ -30,8 +30,6 @@ public class ArrayListBenchmark { @Param({"100", "1000", "10000"}) static int loopTime; - ArrayList> data = new ArrayList<>(); - public ArrayList getRow(int index) { short v1 = (short) index; int v2 = (int) index; @@ -61,17 +59,10 @@ public void getValue(ArrayList rowData) { Integer mayNull = (Integer) rowData.get(6); } - @Setup - public void setup() { - for (int i = 0; i < loopTime; i++) { - data.add(getRow(i)); - } - } - @Benchmark public void arrayListTest() { for (int i = 0; i < loopTime; i++) { - getValue(data.get(i)); + getValue(getRow(i)); } } } diff --git a/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/StreamchunkBenchmark.java b/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/StreamchunkBenchmark.java index 8741044f7b34e..628d1405c8d81 100644 --- a/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/StreamchunkBenchmark.java +++ b/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/StreamchunkBenchmark.java @@ -16,31 +16,37 @@ package com.risingwave.java.binding; +import java.util.ArrayList; +import java.util.Iterator; import java.util.concurrent.TimeUnit; import org.openjdk.jmh.annotations.*; -@Warmup(iterations = 10, time = 1, timeUnit = TimeUnit.MILLISECONDS) -@Measurement(iterations = 20, time = 1, timeUnit = TimeUnit.MILLISECONDS) +@Warmup(iterations = 2, time = 1, timeUnit = TimeUnit.MILLISECONDS, batchSize = 10) +@Measurement(iterations = 5, time = 1, timeUnit = TimeUnit.MILLISECONDS, batchSize = 10) @Fork(value = 1) @BenchmarkMode(org.openjdk.jmh.annotations.Mode.AverageTime) -@OutputTimeUnit(TimeUnit.MICROSECONDS) +@OutputTimeUnit(TimeUnit.MILLISECONDS) @State(org.openjdk.jmh.annotations.Scope.Benchmark) public class StreamchunkBenchmark { @Param({"100", "1000", "10000"}) - static int loopTime; + int loopTime; - String str; - StreamChunkIterator iter; + Iterator iterOfIter; - @Setup(Level.Invocation) + @Setup(Level.Iteration) public void setup() { - str = "i i I f F B i"; - for (int i = 0; i < loopTime; i++) { - String b = i % 2 == 0 ? "f" : "t"; - String n = i % 2 == 0 ? "." : "1"; - str += String.format("\n + %d %d %d %d.0 %d.0 %s %s", i, i, i, i, i, b, n); + var iterList = new ArrayList(); + for (int iterI = 0; iterI < 10; iterI++) { + String str = "i i I f F B i"; + for (int i = 0; i < loopTime; i++) { + String b = i % 2 == 0 ? "f" : "t"; + String n = i % 2 == 0 ? "." : "1"; + str += String.format("\n + %d %d %d %d.0 %d.0 %s %s", i, i, i, i, i, b, n); + } + var iter = new StreamChunkIterator(str); + iterList.add(iter); } - iter = new StreamChunkIterator(str); + iterOfIter = iterList.iterator(); } public void getValue(StreamChunkRow row) { @@ -55,6 +61,10 @@ public void getValue(StreamChunkRow row) { @Benchmark public void streamchunkTest() { + if (!iterOfIter.hasNext()) { + throw new RuntimeException("too few prepared iter"); + } + var iter = iterOfIter.next(); int count = 0; while (true) { try (StreamChunkRow row = iter.next()) { diff --git a/src/java_binding/make-java-binding.toml b/src/java_binding/make-java-binding.toml index 957cec9c762f5..af76a18ba9b45 100644 --- a/src/java_binding/make-java-binding.toml +++ b/src/java_binding/make-java-binding.toml @@ -15,7 +15,7 @@ script = ''' #!/usr/bin/env bash set -ex cd java -mvn install --no-transfer-progress --pl java-binding-integration-test --am -DskipTests=true -Dmaven.javadoc.skip=true +mvn install --no-transfer-progress --pl java-binding-integration-test --am -DskipTests=true -Dmaven.javadoc.skip mvn dependency:copy-dependencies --no-transfer-progress --pl java-binding-integration-test ''' @@ -98,9 +98,6 @@ cd ${RISINGWAVE_ROOT}/java [tasks.run-java-binding-stream-chunk-benchmark] category = "RiseDev - Java Binding" description = "Run the java binding stream chunk benchmark" -dependencies = [ - "build-java-binding", -] script = ''' #!/usr/bin/env bash set -ex @@ -109,10 +106,10 @@ RISINGWAVE_ROOT=$(git rev-parse --show-toplevel) cd ${RISINGWAVE_ROOT}/java -mvn install --pl java-binding-benchmark --am -DskipTests=true -Dmaven.javadoc.skip=true +mvn install --pl java-binding-benchmark --am -DskipTests=true -Dmaven.javadoc.skip mvn dependency:copy-dependencies --pl java-binding-benchmark -java -cp "java-binding-benchmark/target/dependency/*:java-binding-benchmark/target/java-binding-benchmark-1.0-SNAPSHOT.jar" \ +java -cp "java-binding-benchmark/target/dependency/*:java-binding-benchmark/target/java-binding-benchmark-0.1.0-SNAPSHOT.jar" \ com.risingwave.java.binding.BenchmarkRunner '''