From ccad6f4b5b18267c124705a41d71f1217bc1d212 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Fri, 16 Aug 2024 14:25:27 +0800 Subject: [PATCH 01/26] refactor: deprecate old s3 and use use s3_v2 as default (#17963) Signed-off-by: tabversion Signed-off-by: tabVersion Co-authored-by: tabversion --- ci/workflows/main-cron.yml | 48 +++++++++---------- e2e_test/s3/fs_parquet_source.py | 2 +- e2e_test/s3/fs_source_batch.py | 4 +- e2e_test/s3/fs_source_v2.py | 2 +- .../kinesis-s3-source/create_source.sql | 2 +- src/frontend/src/handler/create_source.rs | 27 +++++++---- 6 files changed, 46 insertions(+), 39 deletions(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index f6a6824cc9cd..34155618bf9a 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -464,7 +464,7 @@ steps: timeout_in_minutes: 10 retry: *auto-retry - - label: "S3_v2 source check on AWS (json parser)" + - label: "S3 source check on AWS (json parser)" key: "s3-v2-source-check-aws-json-parser" command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_v2.py -t json" if: | @@ -486,7 +486,7 @@ steps: timeout_in_minutes: 25 retry: *auto-retry - - label: "S3_v2 source new file check on AWS (json)" + - label: "S3 source new file check on AWS (json)" key: "s3-v2-source-new-file-check-aws" command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_v2_new_file.py" if: | @@ -508,7 +508,7 @@ steps: timeout_in_minutes: 25 retry: *auto-retry - - label: "S3_v2 source check on parquet file" + - label: "S3 source check on parquet file" key: "s3-v2-source-check-parquet-file" command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_parquet_source.py" if: | @@ -530,7 +530,7 @@ steps: timeout_in_minutes: 25 retry: *auto-retry - - label: "S3_v2 source batch read on AWS (json parser)" + - label: "S3 source batch read on AWS (json parser)" key: "s3-v2-source-batch-read-check-aws-json-parser" command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_batch.py -t json" if: | @@ -552,7 +552,7 @@ steps: timeout_in_minutes: 25 retry: *auto-retry - - label: "S3_v2 source check on AWS (csv parser)" + - label: "S3 source check on AWS (csv parser)" key: "s3-v2-source-check-aws-csv-parser" command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_v2.py -t csv_without_header" if: | @@ -881,25 +881,25 @@ steps: timeout_in_minutes: 10 retry: *auto-retry -# FIXME(xxhZs): https://github.com/risingwavelabs/risingwave/issues/17855 -# - label: "end-to-end cassandra sink test" -# key: "e2e-cassandra-sink-tests" -# command: "ci/scripts/e2e-cassandra-sink-test.sh -p ci-release" -# if: | -# !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null -# || build.pull_request.labels includes "ci/run-e2e-cassandra-sink-tests" -# || build.env("CI_STEPS") =~ /(^|,)e2e-cassandra-sink-tests?(,|$$)/ -# depends_on: -# - "build" -# - "build-other" -# plugins: -# - docker-compose#v5.1.0: -# run: sink-test-env -# config: ci/docker-compose.yml -# mount-buildkite-agent: true -# - ./ci/plugins/upload-failure-logs -# timeout_in_minutes: 10 -# retry: *auto-retry + # FIXME(xxhZs): https://github.com/risingwavelabs/risingwave/issues/17855 + # - label: "end-to-end cassandra sink test" + # key: "e2e-cassandra-sink-tests" + # command: "ci/scripts/e2e-cassandra-sink-test.sh -p ci-release" + # if: | + # !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null + # || build.pull_request.labels includes "ci/run-e2e-cassandra-sink-tests" + # || build.env("CI_STEPS") =~ /(^|,)e2e-cassandra-sink-tests?(,|$$)/ + # depends_on: + # - "build" + # - "build-other" + # plugins: + # - docker-compose#v5.1.0: + # run: sink-test-env + # config: ci/docker-compose.yml + # mount-buildkite-agent: true + # - ./ci/plugins/upload-failure-logs + # timeout_in_minutes: 10 + # retry: *auto-retry - label: "end-to-end clickhouse sink test" key: "e2e-clickhouse-sink-tests" diff --git a/e2e_test/s3/fs_parquet_source.py b/e2e_test/s3/fs_parquet_source.py index 64060928c775..9422e1403e12 100644 --- a/e2e_test/s3/fs_parquet_source.py +++ b/e2e_test/s3/fs_parquet_source.py @@ -63,7 +63,7 @@ def _table(): test_timestamp timestamp, test_timestamptz timestamptz, ) WITH ( - connector = 's3_v2', + connector = 's3', match_pattern = '*.parquet', s3.region_name = '{config['S3_REGION']}', s3.bucket_name = '{config['S3_BUCKET']}', diff --git a/e2e_test/s3/fs_source_batch.py b/e2e_test/s3/fs_source_batch.py index 9f8da63533a6..fc09b0ef4b51 100644 --- a/e2e_test/s3/fs_source_batch.py +++ b/e2e_test/s3/fs_source_batch.py @@ -69,7 +69,7 @@ def _encode(): sex int, mark int, ) WITH ( - connector = 's3_v2', + connector = 's3', match_pattern = '{prefix}*.{fmt}', s3.region_name = '{config['S3_REGION']}', s3.bucket_name = '{config['S3_BUCKET']}', @@ -136,7 +136,7 @@ def _encode(): sex int, mark int, ) WITH ( - connector = 's3_v2', + connector = 's3', match_pattern = '{prefix}*.{fmt}', s3.region_name = '{config['S3_REGION']}', s3.bucket_name = '{config['S3_BUCKET']}', diff --git a/e2e_test/s3/fs_source_v2.py b/e2e_test/s3/fs_source_v2.py index 6706d4b6d4a9..5cf35b1341ba 100644 --- a/e2e_test/s3/fs_source_v2.py +++ b/e2e_test/s3/fs_source_v2.py @@ -70,7 +70,7 @@ def _encode(): sex int, mark int, ) WITH ( - connector = 's3_v2', + connector = 's3', match_pattern = '{prefix}*.{fmt}', s3.region_name = '{config['S3_REGION']}', s3.bucket_name = '{config['S3_BUCKET']}', diff --git a/integration_tests/kinesis-s3-source/create_source.sql b/integration_tests/kinesis-s3-source/create_source.sql index c4650f22d822..902eccdf1950 100644 --- a/integration_tests/kinesis-s3-source/create_source.sql +++ b/integration_tests/kinesis-s3-source/create_source.sql @@ -16,7 +16,7 @@ CREATE SOURCE ad_click ( bid_id BIGINT, click_timestamp TIMESTAMPTZ ) WITH ( - connector = 's3_v2', + connector = 's3', s3.region_name = 'us-east-1', s3.bucket_name = 'ad-click', s3.credentials.access = 'test', diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 1c9d0b589342..9c7a264816a6 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -52,6 +52,7 @@ use risingwave_connector::source::datagen::DATAGEN_CONNECTOR; use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR; use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType}; use risingwave_connector::source::test_source::TEST_CONNECTOR; +pub use risingwave_connector::source::UPSTREAM_SOURCE_KEY; use risingwave_connector::source::{ ConnectorProperties, GCS_CONNECTOR, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, MQTT_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, OPENDAL_S3_CONNECTOR, @@ -90,8 +91,6 @@ use crate::utils::{ }; use crate::{bind_data_type, build_graph, OptimizerContext, WithOptions, WithOptionsSecResolved}; -pub(crate) const UPSTREAM_SOURCE_KEY: &str = "connector"; - /// Map a JSON schema to a relational schema async fn extract_json_table_schema( schema_config: &Option<(AstString, bool)>, @@ -1119,10 +1118,25 @@ pub fn validate_compatibility( source_schema: &ConnectorSchema, props: &mut BTreeMap, ) -> Result<()> { - let connector = props + let mut connector = props .get_connector() .ok_or_else(|| RwError::from(ProtocolError("missing field 'connector'".to_string())))?; + if connector == OPENDAL_S3_CONNECTOR { + // reject s3_v2 creation + return Err(RwError::from(Deprecated( + OPENDAL_S3_CONNECTOR.to_string(), + S3_CONNECTOR.to_string(), + ))); + } + if connector == S3_CONNECTOR { + // S3 connector is deprecated, use OPENDAL_S3_CONNECTOR instead + // do s3 -> s3_v2 migration + let entry = props.get_mut(UPSTREAM_SOURCE_KEY).unwrap(); + *entry = OPENDAL_S3_CONNECTOR.to_string(); + connector = OPENDAL_S3_CONNECTOR.to_string(); + } + let compatible_formats = CONNECTORS_COMPATIBLE_FORMATS .get(&connector) .ok_or_else(|| { @@ -1152,13 +1166,6 @@ pub fn validate_compatibility( } } - if connector == S3_CONNECTOR { - return Err(RwError::from(Deprecated( - S3_CONNECTOR.to_string(), - OPENDAL_S3_CONNECTOR.to_string(), - ))); - } - let compatible_encodes = compatible_formats .get(&source_schema.format) .ok_or_else(|| { From 7b807cf7bf8f44bf02741fd33a57f30a1fbafd68 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 16 Aug 2024 15:46:47 +0800 Subject: [PATCH 02/26] feat(stream): add approx percentile cache for global approx percentile (#18026) --- .../estimate_size/src/collections/btreemap.rs | 2 +- .../impl/src/aggregate/approx_percentile.rs | 6 + .../src/executor/approx_percentile/global.rs | 174 +--------- .../approx_percentile/global_state.rs | 328 ++++++++++++++++++ .../src/executor/approx_percentile/mod.rs | 1 + ...3cc5911-e5e7-4de0-aff9-30c82b2d40c6.sqlite | Bin 0 -> 294912 bytes 6 files changed, 348 insertions(+), 163 deletions(-) create mode 100644 src/stream/src/executor/approx_percentile/global_state.rs create mode 100644 stest-73cc5911-e5e7-4de0-aff9-30c82b2d40c6.sqlite diff --git a/src/common/estimate_size/src/collections/btreemap.rs b/src/common/estimate_size/src/collections/btreemap.rs index f48a78715f69..af9ab3471ace 100644 --- a/src/common/estimate_size/src/collections/btreemap.rs +++ b/src/common/estimate_size/src/collections/btreemap.rs @@ -44,7 +44,7 @@ impl EstimatedBTreeMap { self.inner.is_empty() } - pub fn iter(&self) -> impl Iterator { + pub fn iter(&self) -> impl DoubleEndedIterator { self.inner.iter() } diff --git a/src/expr/impl/src/aggregate/approx_percentile.rs b/src/expr/impl/src/aggregate/approx_percentile.rs index 1c5df73bc5c0..33e2a9969cdc 100644 --- a/src/expr/impl/src/aggregate/approx_percentile.rs +++ b/src/expr/impl/src/aggregate/approx_percentile.rs @@ -91,9 +91,15 @@ impl ApproxPercentile { } else if non_neg { let count = state.pos_buckets.entry(bucket_id).or_insert(0); *count -= 1; + if *count == 0 { + state.pos_buckets.remove(&bucket_id); + } } else { let count = state.neg_buckets.entry(bucket_id).or_insert(0); *count -= 1; + if *count == 0 { + state.neg_buckets.remove(&bucket_id); + } } state.count -= 1; } diff --git a/src/stream/src/executor/approx_percentile/global.rs b/src/stream/src/executor/approx_percentile/global.rs index 9434ccf05d5a..2ccff36c4739 100644 --- a/src/stream/src/executor/approx_percentile/global.rs +++ b/src/stream/src/executor/approx_percentile/global.rs @@ -12,13 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use core::ops::Bound; - -use risingwave_common::array::Op; -use risingwave_common::row::RowExt; -use risingwave_common::types::ToOwnedDatum; -use risingwave_storage::store::PrefetchOptions; - +use super::global_state::GlobalApproxPercentileState; use crate::executor::prelude::*; pub struct GlobalApproxPercentileExecutor { @@ -27,10 +21,7 @@ pub struct GlobalApproxPercentileExecutor { pub quantile: f64, pub base: f64, pub chunk_size: usize, - /// Used for the approx percentile buckets. - pub bucket_state_table: StateTable, - /// Used for the approx percentile count. - pub count_state_table: StateTable, + pub state: GlobalApproxPercentileState, } impl GlobalApproxPercentileExecutor { @@ -43,186 +34,45 @@ impl GlobalApproxPercentileExecutor { bucket_state_table: StateTable, count_state_table: StateTable, ) -> Self { + let global_state = + GlobalApproxPercentileState::new(quantile, base, bucket_state_table, count_state_table); Self { _ctx, input, quantile, base, chunk_size, - bucket_state_table, - count_state_table, + state: global_state, } } /// TODO(kwannoel): Include cache later. #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(self) { - let mut bucket_state_table = self.bucket_state_table; - let mut count_state_table = self.count_state_table; + // Initialize state let mut input_stream = self.input.execute(); - - // Initialize state tables. let first_barrier = expect_first_barrier(&mut input_stream).await?; - bucket_state_table.init_epoch(first_barrier.epoch); - count_state_table.init_epoch(first_barrier.epoch); + let mut state = self.state; + state.init(first_barrier.epoch).await?; yield Message::Barrier(first_barrier); // Get row count state, and row_count. - let mut row_count_state = count_state_table.get_row(&[Datum::None; 0]).await?; - let mut row_count = if let Some(row) = row_count_state.as_ref() { - row.datum_at(0).unwrap().into_int64() - } else { - 0 - }; - - // Get prev output, based on the current state. - let mut prev_output = Self::get_output( - &bucket_state_table, - row_count as u64, - self.quantile, - self.base, - ) - .await?; - #[for_await] for message in input_stream { match message? { Message::Chunk(chunk) => { - for (_, row) in chunk.rows() { - // Decoding - let sign_datum = row.datum_at(0); - let bucket_id_datum = row.datum_at(1); - let delta_datum = row.datum_at(2); - let delta: i32 = delta_datum.unwrap().into_int32(); - - // Updates - row_count = row_count.checked_add(delta as i64).unwrap(); - - let pk = row.project(&[0, 1]); - let old_row = bucket_state_table.get_row(pk).await?; - let old_bucket_row_count: i64 = if let Some(row) = old_row.as_ref() { - row.datum_at(2).unwrap().into_int64() - } else { - 0 - }; - - let new_value = old_bucket_row_count.checked_add(delta as i64).unwrap(); - let new_value_datum = Datum::from(ScalarImpl::Int64(new_value)); - let new_row = &[ - sign_datum.to_owned_datum(), - bucket_id_datum.map(|d| d.into()), - new_value_datum, - ]; - - if old_row.is_none() { - bucket_state_table.insert(new_row); - } else { - bucket_state_table.update(old_row, new_row); - } - } + state.apply_chunk(chunk)?; } Message::Barrier(barrier) => { - // We maintain an invariant, iff row_count_state is none, - // we haven't pushed any data to downstream. - // Naturally, if row_count_state is some, - // we have pushed data to downstream. - let new_output = Self::get_output( - &bucket_state_table, - row_count as u64, - self.quantile, - self.base, - ) - .await?; - let percentile_chunk = if row_count_state.is_none() { - StreamChunk::from_rows( - &[(Op::Insert, &[new_output.clone()])], - &[DataType::Float64], - ) - } else { - StreamChunk::from_rows( - &[ - (Op::UpdateDelete, &[prev_output.clone()]), - (Op::UpdateInsert, &[new_output.clone()]), - ], - &[DataType::Float64], - ) - }; - prev_output = new_output; - yield Message::Chunk(percentile_chunk); - - let new_row_count_state = &[Datum::from(ScalarImpl::Int64(row_count))]; - if let Some(row_count_state) = row_count_state { - count_state_table.update(row_count_state, new_row_count_state); - } else { - count_state_table.insert(new_row_count_state); - } - row_count_state = Some(new_row_count_state.into_owned_row()); - count_state_table.commit(barrier.epoch).await?; - - bucket_state_table.commit(barrier.epoch).await?; - + let output = state.get_output(); + yield Message::Chunk(output); + state.commit(barrier.epoch).await?; yield Message::Barrier(barrier); } Message::Watermark(_) => {} } } } - - /// We have these scenarios to consider, based on row count state. - /// 1. We have no row count state, this means it's the bootstrap init for this executor. - /// Output NULL as an INSERT. Persist row count state=0. - /// 2. We have row count state. - /// Output UPDATE (`old_state`, `new_state`) to downstream. - async fn get_output( - bucket_state_table: &StateTable, - row_count: u64, - quantile: f64, - base: f64, - ) -> StreamExecutorResult { - let quantile_count = (row_count as f64 * quantile).floor() as u64; - let mut acc_count = 0; - let neg_bounds: (Bound, Bound) = ( - Bound::Unbounded, - Bound::Excluded([Datum::from(ScalarImpl::Int16(0))].to_owned_row()), - ); - let non_neg_bounds: (Bound, Bound) = ( - Bound::Included([Datum::from(ScalarImpl::Int16(0))].to_owned_row()), - Bound::Unbounded, - ); - // Just iterate over the singleton vnode. - // TODO(kwannoel): Should we just use separate state tables for - // positive and negative counts? - // Reverse iterator is not as efficient. - #[for_await] - for keyed_row in bucket_state_table - .rev_iter_with_prefix(&[Datum::None; 0], &neg_bounds, PrefetchOptions::default()) - .await? - .chain( - bucket_state_table - .iter_with_prefix( - &[Datum::None; 0], - &non_neg_bounds, - PrefetchOptions::default(), - ) - .await?, - ) - { - let row = keyed_row?.into_owned_row(); - let count = row.datum_at(2).unwrap().into_int64(); - acc_count += count as u64; - if acc_count > quantile_count { - let sign = row.datum_at(0).unwrap().into_int16(); - if sign == 0 { - return Ok(Datum::from(ScalarImpl::Float64(0.0.into()))); - } - let bucket_id = row.datum_at(1).unwrap().into_int32(); - let percentile_value = sign as f64 * 2.0 * base.powi(bucket_id) / (base + 1.0); - let percentile_datum = Datum::from(ScalarImpl::Float64(percentile_value.into())); - return Ok(percentile_datum); - } - } - Ok(Datum::None) - } } impl Execute for GlobalApproxPercentileExecutor { diff --git a/src/stream/src/executor/approx_percentile/global_state.rs b/src/stream/src/executor/approx_percentile/global_state.rs new file mode 100644 index 000000000000..790d89699e78 --- /dev/null +++ b/src/stream/src/executor/approx_percentile/global_state.rs @@ -0,0 +1,328 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::{BTreeMap, Bound}; +use std::mem; + +use risingwave_common::array::Op; +use risingwave_common::bail; +use risingwave_common::row::Row; +use risingwave_common::types::{Datum, ToOwnedDatum}; +use risingwave_common::util::epoch::EpochPair; +use risingwave_storage::store::PrefetchOptions; +use risingwave_storage::StateStore; + +use crate::executor::prelude::*; +use crate::executor::StreamExecutorResult; + +/// The global approx percentile state. +pub struct GlobalApproxPercentileState { + quantile: f64, + base: f64, + row_count: i64, + bucket_state_table: StateTable, + count_state_table: StateTable, + cache: BucketTableCache, + last_output: Option, + output_changed: bool, +} + +// Initialization +impl GlobalApproxPercentileState { + pub fn new( + quantile: f64, + base: f64, + bucket_state_table: StateTable, + count_state_table: StateTable, + ) -> Self { + Self { + quantile, + base, + row_count: 0, + bucket_state_table, + count_state_table, + cache: BucketTableCache::new(), + last_output: None, + output_changed: false, + } + } + + pub async fn init(&mut self, init_epoch: EpochPair) -> StreamExecutorResult<()> { + // Init state tables. + self.count_state_table.init_epoch(init_epoch); + self.bucket_state_table.init_epoch(init_epoch); + + // Refill row_count + let row_count_state = self.get_row_count_state().await?; + let row_count = Self::decode_row_count(&row_count_state)?; + self.row_count = row_count; + tracing::debug!(?row_count, "recovered row_count"); + + // Refill cache + self.refill_cache().await?; + + // Update the last output downstream + let last_output = if row_count_state.is_none() { + None + } else { + Some(self.cache.get_output(row_count, self.quantile, self.base)) + }; + tracing::debug!(?last_output, "recovered last_output"); + self.last_output = last_output; + Ok(()) + } + + async fn refill_cache(&mut self) -> StreamExecutorResult<()> { + let bounds: (Bound, Bound) = (Bound::Unbounded, Bound::Unbounded); + #[for_await] + for keyed_row in self + .bucket_state_table + .iter_with_prefix(&[Datum::None; 0], &bounds, PrefetchOptions::default()) + .await? + { + let row = keyed_row?.into_owned_row(); + let sign = row.datum_at(0).unwrap().into_int16(); + let bucket_id = row.datum_at(1).unwrap().into_int32(); + let count = row.datum_at(2).unwrap().into_int64(); + match sign { + -1 => { + self.cache.neg_buckets.insert(bucket_id, count as i64); + } + 0 => { + self.cache.zeros = count as i64; + } + 1 => { + self.cache.pos_buckets.insert(bucket_id, count as i64); + } + _ => { + bail!("Invalid sign: {}", sign); + } + } + } + Ok(()) + } + + async fn get_row_count_state(&self) -> StreamExecutorResult> { + self.count_state_table.get_row(&[Datum::None; 0]).await + } + + fn decode_row_count(row_count_state: &Option) -> StreamExecutorResult { + if let Some(row) = row_count_state.as_ref() { + let Some(datum) = row.datum_at(0) else { + bail!("Invalid row count state: {:?}", row) + }; + Ok(datum.into_int64()) + } else { + Ok(0) + } + } +} + +// Update +impl GlobalApproxPercentileState { + pub fn apply_chunk(&mut self, chunk: StreamChunk) -> StreamExecutorResult<()> { + // Op is ignored here, because we only check the `delta` column inside the row. + // The sign of the `delta` column will tell us if we need to decrease or increase the + // count of the bucket. + for (_op, row) in chunk.rows() { + debug_assert_eq!(_op, Op::Insert); + self.apply_row(row)?; + } + Ok(()) + } + + pub fn apply_row(&mut self, row: impl Row) -> StreamExecutorResult<()> { + // Decoding + let sign_datum = row.datum_at(0); + let sign = sign_datum.unwrap().into_int16(); + let sign_datum = sign_datum.to_owned_datum(); + let bucket_id_datum = row.datum_at(1); + let bucket_id = bucket_id_datum.unwrap().into_int32(); + let bucket_id_datum = bucket_id_datum.to_owned_datum(); + let delta_datum = row.datum_at(2); + let delta: i32 = delta_datum.unwrap().into_int32(); + + if delta == 0 { + return Ok(()); + } + + self.output_changed = true; + + // Updates + self.row_count = self.row_count.checked_add(delta as i64).unwrap(); + tracing::debug!("updated row_count: {}", self.row_count); + + let (is_new_entry, old_count, new_count) = match sign { + -1 => { + let count_entry = self.cache.neg_buckets.get(&bucket_id).copied(); + let old_count = count_entry.unwrap_or(0); + let new_count = old_count.checked_add(delta as i64).unwrap(); + let is_new_entry = count_entry.is_none(); + if new_count != 0 { + self.cache.neg_buckets.insert(bucket_id, new_count); + } else { + self.cache.neg_buckets.remove(&bucket_id); + } + (is_new_entry, old_count, new_count) + } + 0 => { + let old_count = self.cache.zeros; + let new_count = old_count.checked_add(delta as i64).unwrap(); + let is_new_entry = old_count == 0; + if new_count != 0 { + self.cache.zeros = new_count; + } + (is_new_entry, old_count, new_count) + } + 1 => { + let count_entry = self.cache.pos_buckets.get(&bucket_id).copied(); + let old_count = count_entry.unwrap_or(0); + let new_count = old_count.checked_add(delta as i64).unwrap(); + let is_new_entry = count_entry.is_none(); + if new_count != 0 { + self.cache.pos_buckets.insert(bucket_id, new_count); + } else { + self.cache.pos_buckets.remove(&bucket_id); + } + (is_new_entry, old_count, new_count) + } + _ => bail!("Invalid sign: {}", sign), + }; + + let old_row = &[ + sign_datum.clone(), + bucket_id_datum.clone(), + Datum::from(ScalarImpl::Int64(old_count)), + ]; + if new_count == 0 && !is_new_entry { + self.bucket_state_table.delete(old_row); + } else if new_count > 0 { + let new_row = &[ + sign_datum, + bucket_id_datum, + Datum::from(ScalarImpl::Int64(new_count)), + ]; + if is_new_entry { + self.bucket_state_table.insert(new_row); + } else { + self.bucket_state_table.update(old_row, new_row); + } + } else { + bail!("invalid state, new_count = 0 and is_new_entry is true") + } + + Ok(()) + } + + pub async fn commit(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { + // Commit row count state. + let row_count_datum = Datum::from(ScalarImpl::Int64(self.row_count)); + let row_count_row = &[row_count_datum]; + let last_row_count_state = self.count_state_table.get_row(&[Datum::None; 0]).await?; + match last_row_count_state { + None => self.count_state_table.insert(row_count_row), + Some(last_row_count_state) => self + .count_state_table + .update(last_row_count_state, row_count_row), + } + self.count_state_table.commit(epoch).await?; + self.bucket_state_table.commit(epoch).await?; + Ok(()) + } +} + +// Read +impl GlobalApproxPercentileState { + pub fn get_output(&mut self) -> StreamChunk { + let last_output = mem::take(&mut self.last_output); + let new_output = if !self.output_changed { + tracing::debug!("last_output: {:#?}", last_output); + last_output.clone().flatten() + } else { + self.cache + .get_output(self.row_count, self.quantile, self.base) + }; + self.last_output = Some(new_output.clone()); + let output_chunk = match last_output { + None => StreamChunk::from_rows(&[(Op::Insert, &[new_output])], &[DataType::Float64]), + Some(last_output) if !self.output_changed => StreamChunk::from_rows( + &[ + (Op::UpdateDelete, &[last_output.clone()]), + (Op::UpdateInsert, &[last_output]), + ], + &[DataType::Float64], + ), + Some(last_output) => StreamChunk::from_rows( + &[ + (Op::UpdateDelete, &[last_output.clone()]), + (Op::UpdateInsert, &[new_output.clone()]), + ], + &[DataType::Float64], + ), + }; + tracing::debug!("get_output: {:#?}", output_chunk,); + self.output_changed = false; + output_chunk + } +} + +type Count = i64; +type BucketId = i32; + +type BucketMap = BTreeMap; + +/// Keeps the entire bucket state table contents in-memory. +struct BucketTableCache { + neg_buckets: BucketMap, + zeros: Count, // If Count is 0, it means this bucket has not be inserted into before. + pos_buckets: BucketMap, +} + +impl BucketTableCache { + pub fn new() -> Self { + Self { + neg_buckets: BucketMap::new(), + zeros: 0, + pos_buckets: BucketMap::new(), + } + } + + pub fn get_output(&self, row_count: i64, quantile: f64, base: f64) -> Datum { + let quantile_count = (row_count as f64 * quantile).floor() as i64; + let mut acc_count = 0; + for (bucket_id, count) in self.neg_buckets.iter().rev() { + acc_count += count; + if acc_count > quantile_count { + // approx value = -2 * y^i / (y + 1) + let approx_percentile = -2.0 * base.powi(*bucket_id) / (base + 1.0); + let approx_percentile = ScalarImpl::Float64(approx_percentile.into()); + return Datum::from(approx_percentile); + } + } + acc_count += self.zeros; + if acc_count > quantile_count { + return Datum::from(ScalarImpl::Float64(0.0.into())); + } + for (bucket_id, count) in &self.pos_buckets { + acc_count += count; + if acc_count > quantile_count { + // approx value = 2 * y^i / (y + 1) + let approx_percentile = 2.0 * base.powi(*bucket_id) / (base + 1.0); + let approx_percentile = ScalarImpl::Float64(approx_percentile.into()); + return Datum::from(approx_percentile); + } + } + Datum::None + } +} diff --git a/src/stream/src/executor/approx_percentile/mod.rs b/src/stream/src/executor/approx_percentile/mod.rs index 29910d9032e1..8d2c5bdcf454 100644 --- a/src/stream/src/executor/approx_percentile/mod.rs +++ b/src/stream/src/executor/approx_percentile/mod.rs @@ -13,4 +13,5 @@ // limitations under the License. pub mod global; +mod global_state; pub mod local; diff --git a/stest-73cc5911-e5e7-4de0-aff9-30c82b2d40c6.sqlite b/stest-73cc5911-e5e7-4de0-aff9-30c82b2d40c6.sqlite new file mode 100644 index 0000000000000000000000000000000000000000..3b4d6d0e9c94b56d373b21de4aef71c5416cf2b7 GIT binary patch literal 294912 zcmeFa3w#^bb>KY%5FiLX07|AHQLhp7ghYY>AEF-C10o>_GGF3D&sD}lU`UPxV4#@+ zDJttkpsd;#5Wi6j;d;}d~J~e_*CN= zZ>#4k4KKKF)Qvjtacpq>PxhkkM>Uka#*+)tmGyx&YuKd&nlPOf)#RL*5%l+a2FByD z$v8I|J2DjKf_h?*+hKm>rBslUvYI$6D%{w3VmLN_mb)*0wu1|1g^U>FH1UeYjf_rm zBU3{|9o*5;@p$6c2r0ttFw_wpA#OZ=G(H|58Hi7CL3#S3nAD(}piJ{dN4UZGP#oGZ z5Sti?4aT`p;Ye$JVEuY_=`?L~N`OCtDw=;+*xPL;)0UeD%WM18NaEyFyrv0OWp`{u z*9v!FU_Hx7*_3!iy_^Pr=Y_l`(__Ap@FUtQ=6&7kOPw^<@KK7Q`5IAp*OCEmb5%8K`^S43-j9?FJ)(B$W6=BTxffwO2JA_K{F(wcY9TbP@rrB`V@6 z7aH@r3B|J%qOs^u#mjkEKZs?AjW&p7nLCJ|b%123Fxxa$TEi)m6Bf1%XJa$`+RBEgl28pkc2)v_bcc>U^t>)T77-xd9hC z3<~>;`7U%(TTo{m@a}Ks~eArZRh8nt=om80lD+k$OWcM#5P%w-w%2x5L}2 zmCf+Z`u>K2|M7(ckN^@u0!RP}AOR$R1dsp{Kmter3A_dfbhCAxn|pXZBWeO)+|?jQ zozZ=LyZ1)ABi-Gdk>1W|9~a%bucv2UH0lMrSl`ze_#a3e0ABxpjdm0|7g8x6eymUA(^3Y0n^Kc`Rn99(kx#-8VZ)DaPM`hj zb=&;T`wYqcNH-tZ)4R93AAZX+E5Z*ws)Dk}&s-wqcKVzXwsMi)z9_FQ!VkA*c=8*V z8Gc?!=S5O>SBrD1ST@Q>c1QXmz5HB0laZ5`cumTPJp6d#yqG41JDQ!Rjl#QoB7C$5 z8Xw^mF(c25RB`dgHA(62CTG$t9o@r6`nz}c^zt+Lto|c@NjZ~KpxxvbkfkiNp2&n7 zo#%@({U8(F-QCBlQuY%3#3(?aKa%zV;2(eA!JXh42iO)3)oxn*J~`2}7+ zc_~LlcGNpZ4Ut~3GuqqT-MhDBr;^VSj70e!cd_exqhQ_c?w&}Qa`K~=N*1*1mXe?3 zB$C@)#jfv-0{On)UTB>P_{i@e(mti;_0>70Vp}5MBHjHxJv{x%Xyf-BO)SM9ha~E^ zIGx7~u^#CDNPq9%2zZReYEo*W!+C&~>WM}o(4pOtUVv@*3MsITb@uB8fGh3sZh%XE zL73!{@Bg7)P`abRZLtox^(zmwdUt2!a@>I(MEhk$xHUC@l|I_>j%}Y=OUq}E6 zAOR$R1dsp{Kmter2_OL^U=ZkY%IqEQA+&CG-e*YF+&}PdcTU*K)Yv=l?Ql*N%iU_< zpe5uyZ4|uCoq+Vuiw|y@{L*=FdYdX0VIF~kN^@u z0!RP}AOR$R1dzb1n84O8_4Ulg&5pIq!07N;Y+!P9Ji50#9NoPq+#Bw0uN$8l8A*&B z+u^Qfwn8bEl$x51^CP2!akF4gIJ#<&QLwIVi>sblPl_HLA03&Dj|`f{y2IVAUo?t2 zojdAosj<7L(c#$c)QdJ$ue_&N^w<_hZDXQEV~%ZR^(J~Mn;0!N@fbb-_q~(x{m}Qn zd|&l_-uFr0`{58@NB{{S0VIF~kN^@u0!RP}AOR$R1dzbbfk0cmvx}9dFN#TxPl-7Z zempUoTy);!c6PEtQj--vC8;?IRoHpr>wVs&lSqXB1&JBW5+1*XdltF38Fy zQK|Pj;Ipo=$-_F^*t{w#4s!nQ^!+2_`#0Zr;Sau$01`j~NB{{S0VIF~kN^@u0!RP} zAc5B^fqJ)_X=w1Uj`}+DgVXpx&i@_0uQ9%VhBtg60VIF~kN^@u0!RP}AOR$R1dsp{ zKmxBx0)B^wS>FIJCMT=fEIiw>iS;mTG)0kRjr{(f!}l%5_buO#UXuo5qmcj-Kmter z2_OL^fCP{L5zIygW z_Et$6)2|0koE+k!@Z)B^;YjDWq)OS@)55&i84X9HkRqgWa{{*`pS_fo7qX$dktu9w ze6=^wd;11PD+xyLhQ{5<6gD*6RRj0lO=Wi@Q`q3XO#{34MI&VdNz&a^b~iG+U0!BQ z0|R4(VI~HS$A@EOjUkW1tCF<4?k?q=cPZz%JGniq7q0R%46$!822mh35wBuSl_YZh z&-h%d@YR3(_W!?Ek6^t>00|%g zB!C2v01`j~NB{{S0VIF~UO578*2%0)iSy+5|5)FT8TcPxNB{{S0VIF~kN^@u0!RP} zAOR$R1dzb%jey&^vH{Nj$@Tv_-*b%bUwz;3eGWe03ke_rB!C2v01`j~NB{{S0VIF~ zkN^^R)e&fRIhjq$0-qE#AuZ3g)j64{l%0{4jG#$!mRFN=Vn%3oLaE%WS*pR|WY*;J z(`hM5&i|dhM;YH&d>?{8_(B3m00|%gB!C2v01`j~NB{{S0VIF~ehvh-HoBShn^tZb z7>~y$6C=ltj*iEVjgL-^3{FiB1Y;>7r%Cf-a7%-mS!))H57sEP$xTWzx-x62k|J{c zzmk0u<6Gzy5jGZ`UQAzwOL8KJs$_@cOA@ z>3m~-VAn48!HgzMr^N+Xxg;ulPLXq>qAk+Hr9DQG?KO*(B$}vC#5wwq7cN+)FUajE zrb($FCuKEpR#dpL@x*X!{4950{A>polvG~T6j8{)b6$env@EAZAS7<ZdOG;wXa4Wa>N-G@s;oh?7Ufw*fH+hObWli=P3npP7hI6EIiAY$GE8`+NM)?DcayEv zcF+|pg4G9Tm2OiAgGy~=l$$t0T*XK)S7f1Z8(rm+cj0_o=)tKL8X6yQSq7Z$v_89+ z6g|0YoLDA}$^kWfk=GV;m0>anPY475tl=T0uxXA)a4BnCu_Q}bNt1-MbQL^DfRIY# z@nHODY-(td8<-l0$zqb9Obo{-CS${6mZ%j>!rVsu?arj^_3_DcUe&;oc~Mb|?g_0e zDX&$;?vh0wx9RlTJCK!=U0JaQYt1_49#!gxJ5}4zWDqpleTpQ} zcw-I_vtzAGkqWP}OuQ9y5nZ+9VbxSE3Ivlvme+fRPfg#hgxV32NsPUnpu)sO3V3-0r|cXDu7bE9Il*DfNYoPI`J%$kW~t)3Evl zb9gwTNQp|>VV%vUvP^CVjFfpPDRQS`;{(TI<2$g5Q~v|f|K6=@4Ub?KiBj_&rbM&FC>5jULFFEoNzV3uWhzHmR-BD zamDf2#BmS1f@RzP!R>i!gnSrzjP*?A^c|UZv0iVT$HB2H9P`n4u|7Ch2?zDOA}_$a z_`lto{OoErXeksdFVp^J&wD{-^ASNy&W)vo?1-EaZ=?4)mVK=Lm^bEeukh>j``LDA z!5DjvS+P=YM7!AtlGyG6BkDY?pAE81w}WB#u!q?bEHs;C9%Fka==Rbi*&LdHr5X1| zjz-UYoYui5<#axiRrhh{=c65*Vb}%kfp@V@hUs-$m>Az`7~i0UjZ$x^|ARVcBGl0i zRdTMK|C#kP`5gdc9STOzR@3(&tL{TGxmsWP+3sR*jH%h`9b?0Z-M{S+IpO>7#sV?N3y zMkYfaU|BHf^Q;3b`Wk!f+H=o8d-Db73&l@AVjW=RyBxiD^Ss{o+@+Y?;dea7zDf7N zVK@_1^wUW2Lc8x3{c6XYl!9HT(1i|8ABOxTaSuFq3WZ(W9a7ew`k?F45e`~4RK|lb{ER71{cD6orl={$RbYR%% zz+qS|jVEG5iF5Hm?o=Xvn#*)^u?cPhHdY5Fxt-k6@zG&U(_OXbrst#G9nt88^U({B zBI?7^(T2xifjDVWvW zKA1(96U-tVdr2%S9CJV{bF?ift*vgRP+FPo^ZYtlm_5ce>2t`0oL7>|SVFzW4Qrn* zy3AlET`}KS!-Hi7S7n)3A6oWG?EHoE7alyo(y09Ke^|NF?VyIZ$jb7&tf!9reK+fI zlD}s>4r*8Pu&B(6@hf5yt~)8WSsRiyCs|@yT3oT_e2iVAn?03X=9r=<8ir)WJoveO&amYL+|>5A@hoUD2&(eWMp@&G}|0 zmCg8N$>w}N3VSdM3S7dwZ392*hGo9nGC;1wAqD&MU91%+_4kl<()ryN&UasEZu$;W z&(1KO|Jm}+=BW1z4gbc@)VW`t(;w7g61cH`4c)VSOB+2IQfXAt-`CmJ*Fimz?(yj# zZCj$2>oTx+Y1uQkY_nR2PjMfX>~!i&SYrp1f&5 zhK%CwXxXkOWT_%tE6{jFoYCurB-n-(`LvXgO54^I7fon~k?Qa8qDi@;q-<@nU0Ep) zx*xjBRmDgncqC8#U6ZAx_(y4NnD$`PuswvWRp6*-2d0*wV3CM?= zWym>UWgcv=!yOE~oJ~V!l}k!9iZGiIv*nl3={5v8q10zM>kzFqbmxV1UL?g~kNge? z;GB?6rA3~Oaee0Er^Pv8UXtNzTG@RrL9jIi_ZPr;z5?6$S$+;SR*LN_US)vpDGlM; zjRLlW8#EhgudFMTfsK{wgpz`il3+VW(g;|%-Uc-$$&E_%MgqQAc~$BprYMqe(*Q9L zgiG;2R|XLP`h{GqfN&sZNz^m0S3zMZo0Q4zQM@W;FIAy!C^NPMDmu}A3Ai{+?r=z! z5BEx+e~AjM=1-PK?2D-9AYWFXr zMR&}kaDH`tVACe{I!l?9CY#cywRRe%42q(3Xbr$+M5<2$mh0%HN#AakUHctd_h4Iq zUT}Tr0<{jVEz1fI!9GX!#YLt3xZQ4Pc{a7zvcRw!S#ckOE^IAbvN2d{LXA6U$b@7B z%^I;@&TF|mOewYIf;mB*<1;Wd>34AI@y=3jt(L*PD#l!AT2LkQrAnprz*g$u@-b{` zDO)hnWD;2{`L_=%#h$BNfteAC=rljqh*n$(mg9qDnRDZE$uDR_)>QET+U><=niB-K zlv+iXN;mXdc4>h-)nQr2S2&Q>h+?G{AF35$RJhi!L7y4(t~v3heMV=K zxsqHZ;in}HoT#WxypmI3!h_2!S71RFZS>C98|!ITnX=|z z`&Efz&(l6L%hU=?70AoOx`lR87m3_3u2fjb7)u4~dWekM;zF`|kT*sbp<6*PNIF&I z^)XdE3A6T7u^nXm>Z=+%_Pd=3$o2o#o^`8yPx=nGe68hL3)lQyv)a6^>F=A~+H|P# zUmBlm9P$3d`;_;Lx4z-ohGavl=Px`N&pPwW7XKmvB!C2v01`j~uPXv$qZ{DT(prY? zIkn0)J~c7|Uq^JmF&GW*J0I=n?&#_0?da>+ec=JP9ca@n41W%8PPuv0>xAzwj;?=+ z3cp8G9io*{g#)AODm2zDw~6F_nVY^+%MFcd??$6;BwJEuZDV_dM&~Wt_@k1QWeORi zjcb;#k?6WvU#_QgY-n6v)L8fObamh*Dy;14RTUavYFD@2jmBGc^*t3DUusvcT)xKT zb@jlBCS7Igsj#Ei*UKEmzIZ05Epn5vW5u0=Evplacc#nQtL3`H6W$7Cb#JWRu!^x; zsjOic%4+mwr7};2GN-j=w;HjP%Ia$>yKN6vD7)ZpU>YDs?>!YR_GPdeLUez}o{qg8 z(FokV)e-GpZj`Rn4|lM>ME$qy^9yiOnO#2&>See8_HJ;&d3XAk>lqi|MiP7bVZh(I z{deRn7vT1e~!lBlZf?M?BEfn}m3T|OSZ3PE?%jw11@`qYV4Y)-w-aeTjen@-djCJ#9MqE z3As-{dcj4%ZD9N|o>l!Vo*ptX$`2fm58Ow7n*RKS!g$kqCeEy93#oN0TRp6YS=$V) zUTL=aR!{DMXUxRMM`A;b@Zg%tNBF?gP^v$|$9wxn_;_nS!pCbo!smPhTJO?tCBI;? zi6j@g8*bSXQ7fvKrzh%e(-YMmePC%d?FsL)d!qcYBW6!D{JZr8e6fQ3;7oZ>fagC` z8WV8z%8!ZLbVRiW>XdavvwI!mV*<%kXGQkr(osOpp1wd?q6&FY)H1l*!`{RX$oZxp!$@XMLc*pS^yXJ|D5XP)bZ| zLe<2m{h^*!%g_f#l4mNGJt^M$j92nNUPYXzkCQAb23Z;53OpGz15c0F;PKRAPEO8M zXoF&^FeBE^g@;GZOYp$^j66@C4k^pzY1h^VOwQ!f=^`#_vhfhfP$KNUzPN!tEAyea z{;bS=Hc1|CJ(JH~=Qq?}KX2qVl;_e?K|3W7(v!IzT+lqI^(0NOtMmj|YjXuc3 z@#fIPX*p#)y&VoNfd^JTj9f@B2#YEY4Nw;0f$n@()W`!=c~McwbJDAHh?0kDGNng0 z7g3d-{@KBSUx4SRDtT@>JkVTO`6NeTc<`6C#T|EVq`FW1|IHS zT{OtSLx_WR1>`ZcGzGc^97opw=x|BGtRlekaG_@xgQ4u^`as_rwy?I?KI8ejb22>8 zIVUUBokV4O%|?}%*IT5gl9mufq*kIU7%DVu(tACj_c}b;JT1?f0U~KX+*;P__FUTQ zRYq}H1T;c5bi6bR55O&sKQ442=nmYXZ|$`~-`W$l74~nW&+1+ZQB%}K6}plqPe9Lz zny8c?Z?+p#p1oXagmk`2B#~4;lc&w7I_6WNnpC758C^yQ2_53xfr-vqj$Ph5K3bl# zrKyd5S77~m_LpL`UNaV$e}BQEGa2}{fLV;|FH_vMKQ!5-OrlmrXdO0LGkTIiuSgk~J&F&AuKEmWqjnfQ zRi;)wGbyL@nXJmEMR4FMkFmCvg5WCWL`9R#nYw)1GU}^BPE5i>!gtV!86tsoFg_HY zjB^9AiGkQ)ytyU(u0GJ##$KPGb%Zmny_GR}s&_H0LZ_$187V7Sf{n!?jE=0BhVAZcHpy=OG8PpK!yY82KXQ_q zuEGL~{M%qRog`7y$%i{Oy_N~V9IW*CT$Lq;?IG)zL}N{7Lnbd|jpwWr1?5C(J4@>` ztK{uFqTq?rd8#l#nV4r2s(3jsW|N}j_{Pq))N$U@kMwTm#@Q?c3$j{;2y8|q;``I| z^r1!<&M0!`mie-h5j37m*ZK$njT)SHh?*Wjtu?7)5?1}N@}DUdl@+qYfwM6~#^$Nz z(aZGxI~-%>YGCVu2I~kab1dsp{Kmter z2_OL^fCP{L5{&w?UHhvZg;0p;L0VIF~kN^@u0!RP} zAc3DRfen7w0bc_{zIe!MiZCyx$yXN17arjb>Wmt>uKL`YA3E8oRYe}|k{1_G z%uOFlN~6+=iK(koSB9m8n#hj#B?l6_6PI#lP7R#c8-_wxQ>PQ8&|+#a8aa1nB$AL8 zq|~u=Iw?WH?2$R)^hs%G;6yIjGfc|SGDBH7?oQ{=^`z!fnW_BQ?!6k3J9lPoA#qXe z8=bsV{5NwloZNjZGrjxDTu)pGM6AC*J2cRD z^pZR}lsl$sSLaXmU!KX1ojQ{5jdl+o)$SV~oS*9)y0quS!m<9`DIs!tXd$|LP&_`V zojQKyT>jGV@TLPVT+9ckk@T<*|vy$%|J<&rGK7 zOG~LE<3pm-eRV7{-!nARdG1Pf_*g`XClk|?Qg(8>=g7!_7CX6Tdf~{#_*mze-aW}v zlc$B5i^91p^ZeCg(uv9LJ^52T)8o4@F2J`%FP=Kmd+ONykxt_&Ms<7UR>Sm3I!S%F)K``sRJdY?Cgw?R>e(CuKkdc6NnP}{nslR z58ry#h1cGF1>e)Z?wn2L6$Si`hnwdG4YZJYL?J^yU6iNQ-Y&1}K(v9G7Lu3p zImnb1`06g7k`%bzUse{)Z%afX(JsBrWP>|ea8*d;m9#2l_H}g`5<&{<*k?UB9NpU; zj_%$Qj>4aPdwW2mkqyc>cwC{)9wr6fIjmawRK0t10oai*po_Gb<`+43y`*a0uD*bW zfv+&b7ZoTy_ztKF-{qMXBVlOPJucVwR+))hFi@v=_i}Zv11mky35t+T!_YRq z1*;dxO!q~*15Vd2D4{}gNe@!V%yf6(?skW(4>~6~CnhiD;7j%zKcnbhOj(SsV_iLM z1gxUKx1xFSwJ&9!eAir!Mj{a}L@PeKmter2_OL^fCP{L5GsN2WG@92s_p>!NGn#ff&$7$g#b_igt+j>WUFD3HXTokp18I$YzrjOn zvg|O_uXgsa)%FJ5t9nr+m#6WB^F**_;*Z*1H z_ZaveUq}E6AOR$R1dsp{Kmter2_OL^fCP}h>x6*MxwXN%0YJ|G>wW(buK)Yq=l{Jby9FDC1dsp{Kmter2_OL^fCP{L z5*k2OYQwY zL%;t29OL_%@445HfoKU5Kmter2_OL^fCP{L51DhQ4-5Ys z@XrbV>foOX{<$}>kngCSkJtZSU;cpBAOR$R1dsp{Kmter2_OL^fCP{L5-1WVyZ-O= zeV6h5Z{H8#55AB95 z^DucN?UJF3-Q#FS?zs$6Yz!r+jbqjr&%& ze6{8GS}wLknt#~*XU)Ux7hJj9|Mv^s0Y6{kA6k9D!?JFV=T}xAJV3gA%f}eTbI^aF z{TcXr$cpznyiYs4?{)aEUHkU$f8yo~&fpt7F8}`aZf^kE*}$A)3NQTZo2{<<8B=zJ zp7B1KvFXzvaGDw16Ax|J$u>m($KS3!{LcS)+*Am^a;a-R^QB+fFvo)aF8{qn{R~y( z1IIJf82YDZ#`^eJhnd0s$w$`^{a^m$z~RR)R?+{{kF25kxB3qi^*>G3_`T1SxBn*1 z;6}g47WF^zSUb`Gu?JTlemGr4|Ciq0KF79y-|2Cz__ovgPfqVQoF!}{9;bhA`!;VY zVDkdDYo|=TEA*uIGS}W~CUNgsyOZ@s9$tNTvPwNqtOYZMJ$3$lMSV{(*Pb$k4Snxp zuD#Dp;+}O7eY1xhRn$H00Cgjt2z0sGJchYe;-NQhg1I(PRJWYwMg0eh`X-rclcuoI zyea0|l$o@g=56yIE~Q68AjxlQ;5Q;P7}Ab3X%}1oXE1V?}*G zW3K(o6gKqTWUk#blWyBQ&UE2(t3BQo_jt@h=>0E0Mh4Al2jsZzInRIbS(4*~oCbT2 zA>#~2>*>f0cSoTqFTVRa49sqS6l#0xw?6{1Cmi*jOhg~Og(J~Ou~5_>DHa0t^;Eqs z15MID(26}(b4raZ%9xE^QoHo}Zj3~_Xnn9L46f7F{vi@AKIrh$fC05U^8#sNkoP$J zL1VPJjc9T7lQdeq_->G;quI^q(cV0A>Q19}G+y9(U|FFaRti$_35~CmguavsrVf`D6ZMMZ|Dr=*D)JtPD z&ES6H^EX&R<)aV2>G0p(Z~}H+C1BS-ocXuU-e3WX;A!)7#VTxJ+FFH&w^ygzk}HMHk+h= zXqDV)JNbc}&Cw6}KLpD3XO^Ijik;~-vLR^zY!##`4E}x z7=~>kOU-Q?H<0`Z7vyjBZ-D&A->K(2e~0F8SWogt*F*jW|9Z&(!H=Gy^3FHY{PpWd z{(XIrzuv!&wCwMVe4gg7TTAkf4MYAq|5{T2M~(bLG=FV7$sZ6Qf33fr)c>1C{zjVL zzJ}xW7fc9E-u*(F5fB|82e{-yis%^t}t75cn&;hkZ-%@3#N7{6pn`P!Yb601`j~NB{{S z0VIF~kN^@u0!RP}y!-^nx7^_>3&Z?ha2bVR{u5l>V3^;8&31-)2WGF2g(m z8?p>@9X3}P<^k9!Wtaun^kkUJuz|@imtZrJVMN$iWEg(y7I;4k8-5Hk0h@OWGYlJX za4NltgZCJ0kTJ}D*al;mKZ4CJs28@hpkCOxf_h5jkN^@u0!RP}AOR$R1dsp{ zKmsq7!1YyZo#&ULZwy8w!F_KGYO*Gz?-LgXwy*;H`tgY+o>EBIM?XIdRLSB`P>)mb3u<0|hK;_<-NhRGU*^>;Del7a8BT;0<3$00|%gB!C2v z01`j~NB{{S0VIF~kict*z)FWRQqJUrq$bH(UK7+y z9J&7Q@O_E#eFxs~g#?fQ5?3M%y zI}g?e4jf<$v-I|oxqK!gCol10PEOA8n#|9OiVC-v)Ji`xFdmOh#<|JZk)b#jtW_+? z?cjn`XONqgW~Hpgjf_rmBU3{|+}L8@S zczX~=$-6^ggD|z<9gqg@1cHga=n7R#I8@lPr#{fv$1bh3fiSmSrR(p{iTkyI#M)t6D)A9d?}-w>Y-f9XQ{2J0e!`l1Erm4-`)J*9X?DVV54D3{)|x zh?@Ss-)^U#s9Q!$(Kf@-7H37J*6_{>88OId;+0YWAcLO#8j#t;MqKFNj*gDU6URnK zVBmHH_3B6+A#OZ=G(H|58Hi7CL7f*gBPi3n(GhMiJ`{(TG!UB@hz-WMP+?OgF3C+LWj!6)8vV+A~muVtwwwR9CI; zt7sn|tw`DjQT`B#^w*s|^?|Kh*&7MH1M}0x+EmNue#1ofK{uTrmWFmyu?NpDOU#J( zR*kwRSgNr3gWbuiumd+pi8E4GvT!ly2*?936tnb3M{oo=n^Mi`fs6`krEr_12EJtY zA=R1_VZO*}aEm0lfs)KDV8mXZP6h>}5KvDqc3RmIiPr5pf4D^xw`FGr|)7YRMp5a&IBfRUhc@ zXA8qNt89hLw%ZbWs@rEPLyaW8d+QYjZD3gp;zEh=oft{yNxY?zTWO-$(vy$WPxPzf_)?OaWf<|QG` zXXJSjxMevl3fZF1fcQ*4oi-Pb`Wgy~k!4J%u)Cu^uzx?h{HBF=mOx?|5i8=aP18&3CaRLBP%d#Y6~)-n-f$x37nms5z?x7 zl%#Wz&e3!Z(m5GcS{do8Ko$?&!km~DVZ3qLoT!M~RZifNbFu_N)1tN@idkE6HOvi) zGeSPCshlQrw2Ck{A!?eGodp#VP1cmW$Sp|eG$*7NghiE8MUA^ZK0;23`N`9x{Mhl> zM4XQuJ9d9bOJU6_*Z#J`o52_{OV=efYlzM`O}~XT_SO-RWAswtuX;dGW0vMFv_(2{c;tZ@`cEMrMqd*YtDvw;Dq zrzpk?>9oASUsUDnG!K>P8T=)2k;0FWT<(6FeLt+?jQTjpzuvUcb+YZoXh~I`IEJDK z*;z3|mVeVi^3n`+lf?i_HZ)exF0-R{>$%h5OtiDXH;B>>;#C|utw^WEw4`P@d4?wC zb9$?|8AX^SZPDFjTI5o&n9b+XVhXb4OfC78ypT0Cs7aV-O*LU|RN*qRB673f)}-3S z+qZ#EUR)Q@gH06po0-4+)>P5e#i>&EQdn5wO0*SNTi+UvKAl(R7J1@3mY$%A9CawW zAJyicY;`391!uJ?=u(MRLEXj}Ni=oYb?BqL(Cl~h`wEv!Ze{7;NMeS&e_|50z=jhe z$M~_>cx-4WK9rak2Io|{teow=qIn7nPd+NP#`?t;@a#hoF*U_ z^%Y%YLQKP#$Ve^Vp=wS{N;6WDqZ@QIgg{&aPlouW$%>Q|(s~vh8B`4!;3+{9!W!8W%1{CVF+_+m(`&YF9F^C@^%oEc!Y@UwBT-qz_6>u3c*jp?OJ_rX|=H zT!fWrN`wlV|zi(ZM+F9~dtk+=3Ci)qE}|D_Y4EVjuL0HYevb zm?XsHr5thD@RGODwRzX|q$&yu?2+aKZ7#S4x=jgoaJl?+T1vtMp#!9$k%Xy*ISB%x z3R*J2Tz~A$;@826S~46Wo~4L0Fnz(9 zBgEK6n8Q>l13`o%r`5E{;Ce=FX77qA(y`0H9PRB=;Afn{!8-M&NoO)G0e8xI^RcOc z&WU(zeBgNJSZwn6!KByK<11_;=M-uVxbrmJ^0Pv2&m?#dq;v2YCPs+c&{!HM!5v_T zXE-$j$s84c#sUJMD{~p>Fo~n6xwI@_%ICsMQiCfVxUrH>C+2jS0_Rku={z}!NoCTu zabC*e28V|_I8`Qur7U!k257KG$&v(y64U^H$l4_*DukN=-A+d$-75hpeU>ei1CIg+BOVE^ zrmskdRVhWzkzp;t;)g6abkCa+q;wvN&j~r$CQ-u$#^XBBR=5wMstvaaV}mTSEFn9u z3bP^&AdnY>l@0;{q)y85E-t&f8oGypnak(jAM6i{a*9k6bIjvNeqQ@v}T ztq?13f$j{&dE=m(566Sfe-=KlUXJ$lh{^PeYg;EyEZm2 z4G_;zwZ*iUfL?$By_h4RFuh2pd@}67HFDteNquRV&p{u-kjX-1CEZ1Q9a;z70PD># z{9VT2Gh^IsM&KaKBNLNnhvEkdBQ95G^U|9w4dF=?D25WFa#CpZ>Z{qwg zVx%Ov3k-?0JPT{JyNnee3|Dt%^-~fN2#}1@@5D7yKdD24HBCjt_TsR5l(gH1-_9#960nQ`_}EeL^N7 zfLz0h!LE1)Iz>pAF5*?|~XP_R2Jr%JA9uejzK%qM_*lvwfdT=3O*q8@F zA?&jgUnUhAlTrpYZd48;qVBDWFoZK?St_OAL`o)g*_ZJocGawoDkd1{KnWLG9At@h z7Oo+{SR%Vl=C{@L^L{Xj&&<=%P3CBQmBbH>4v)nKCKID0{IT)TsWBeTR`}slbjp^q zBz~vJn1kLfDdea^wYZYiYoy_O_!LKvNY@lDu&%Da^;U>2WX2?68GH=Zqs2CipXL+8 zV?&97#3X-oXlmkkv04C~<;c0?B9|DMh>uTpO^uO(*kxR9D9s`CToBIZlb1L$lcp(I zm_j9pvt(5P4!S^k0cLSnANTQZLxXfhoFfku&{* zgcT6v7U2ASFhIWlA8wQx-`kp;jUVuBYChg5Hw`uYbF6{_+37{t^Wb5R5cYk- zE8#b}^b>yArb@~wau&MQxu0x<8=K$i9PIwVWo|hS=W`w8TfOAE137!i!ai0iy({EA zZQnJAE9?rt0K4^YT_h>M79yXM;ml=mXPdK&)%C!x8=PLH;WC#N3OIYHN^+&clD=c3 zbAW|Ylx#{+Qg9|l&sU*&k+WSbC7-%5nZ`Hq>eWxI2Nl~Uwx8~9~1%LcG{-;yD?nc84(zM0a&)y_UvUCbuu z6gdkIkApT{CL4t6rdH>6mRw$dI|}scCLAHIZ%`JAZzKqeG;(D>13TofQ>AWL<&2uW z15D{XV6c_2o_@8O?$VeyEUY8V(zm}Xd+%#W?c^+8f%9m(G1N|=Lf_Jpb7KRPoPkTQ zVM{j4)(}I=`ksfKitOrk=iZW_?aE)k?k~2yRrWlxt5v)#*S6Dn*6uiT)3w^Aw0pKW zPl7KPSL%4!KdXA3Y-P}yuxqTifU}}jkGOQ=NHco=|)|)1ddz@Vj&`|p}27dql z6^Zkx6bT>!B!C2v01`j~NB{{S0VIF~kie^o0G|K9s=5Okg9MNO5q~{}n@vije>kKmter2_OL^ zfCP{L5e*{7iRYvWZo5peAdED6d2KO-wik=GVvJ~t<* zA}`F&&IoB$JWA3zNatue2kD%g)1-`aRnR0k%Pq`_S#e%eIBiZ;#O*34aLGAY0-WAswtw7A#{XniklX+@{DlR7| zV1$^K)C?!j(4>6MXy=R~%#ya~?lLWMDKRPMb7?Wf%_wrFmV8QH$Ql~dq>zQOx|%RI zs&E-u5xH4#Yf|mv?c1OcZ{xavUP`3xU-ay$5M|+{!@9OszE|=WO z(!Y_!40r#;BqJm2El>?@omAQyQtCnx736JlCS(ptbn)ts1=W~3xX^JoZxxCWjK@lBHz zDJi7&EIKl%8Zy9Bf+mDHAQ0)GLwu1WrUgw@H9hZQv#UQ;Xw_f@szQ>4UtUsWC;^`! zN~E}?oXJRX(*wOK_}FQ>z@GTW7slWJEouP88dx-9xS zL0@=I%cKuVO|D&Q3!!;Qm8PY%q%HCpIVGm_;7xkzRAM4=Br%kjJj)M{4#sK!zIXSPvBq1g*<%r9Mm%NRx&AYBARZ&oqb9_$F=7L+G+mv7j zm&;G5r6f!cIzSp4NtjxglOPbPpd|x6E5Lx&rD5EY83}?VCr@7_U9y7?sSZkz(-w0O zRA%y7O0R<}g*tKoH@wLWWqQ&4KF~wVAyurbx#w19P;u zOUY-6H+3fE?2I(qnM_N-opRoMY-*r$A|4wbINmuHn>>Cn>2>w^3Y#**6<*B&cb=DWiA7=t;A8(Tw0bd<#XXB zslgQw+*nDc6LY#ufpaR-bROJ?Pi4}!abC*e28V|_I8`Qur7U!k257YY4&;D;N>oE( zZUk!9*sRxmHA)P0y&t+M(j1DlZf~vqCvigEQQWg#~ z)Dgm+9hHp-3n99Rv!Vhf&N^M2+OGHM6EX<_LSB<&97G`K6hT=eUIc-cn->&`j#TI~ zP!GeNidX}W2*Dmu=#C7wTO*YoTu2x;=0Q*hD2OkU3XMrABSJjpAR_ACx(Gu!Lzbmd z3QnYCQkQ)hPhwZi>ZoFZfew^#p~XR#XlIi!-D@IGf}^nzm{nleOHSkX%sdUo~fpv8SuD3#LAu}ck%iv?M9xb+E{4}2!9veywBqsTzLsJvSi`4??EJw~A7rDg9 zM0|X*Yif)P#IC{kP#hL_r8$J23r+*1OB|U=)08Ysp%TPdvZ??FT_C*xvpB4A=E&5X zfCT{U$pwg&D$Mk>9z`VUDVR2?7i{Ce6kWH-nSMdS3J7uwilm8Qdj9V?&G^QAtu4RX za`p*X&fz^w?Z4A*U4 z**v`nvpWx~VOZhJiB}4HI$gWkmWK5S4whv}IR#O7YVzowi8*;eUjgfxohpeY$vIg5 zBsB4 z-&94$REo&K1_X>6K|c?f7ND3J0qAK`c&WF;wLkD+4}<~ZoK%Ij!9r4%^AKjOYe_u= zmN+CM%nj%#A+QLLv+xE1jJr>mxdbOd`fh-(F4bWMGj3vZ3btDA*T=0MmB@OXsDJxRIcevw_+*!G0Mm19w9Uf0JEgC?$6=Bf92_czK$XlArGkZ>J6)&SmR!KQ zcvj2H5aRd+*oA=oo}>V$Mtn|&h`czHPp1dSF`NXDV+t8uLW9*VSz*yEP+!!<5lsM? zXo4Qn7w|!%2F}QWikJk?BA@QpKk_q@qH28h{vd2!l~U7!o)|8?A>`_6y}m{_8ct|n z=1Rj3qSi@PAeL%L4o7pi9SU)HSlxni!sML_(MF5VRNHnCaTnNbfh`if|4sTZY{*Fp zaWR+_gcPh+U_KeUs2P%+g$^#g$pnjXHzgLV97xjW>78hJop0}i!bL&d}1;- zG8h{l>>L;!fn^HW2I_baFK7MZqo?6~d|+xE zHZ~51xv4C80A2Nr!%j$c_Ovh$C%Wd=25~>SWYUaHl5m0B>N?+AxSUKSjo8O82(Tt9 zjxG2EBabKB0TB3mB1b_5b$QZ}B;?TiJe*iU39=%G*hWGr_11yG0n2bP{F|PIQAyq3 zA!bjDM4Pc>BSuKtK4T!k3F*2muC2bMO=LM`wgdLBUcQs5q0xc+Iwupu@zJTtgLh7F zaFuX zku(X`f}}?XAkuwsxq|M-=w|{Uh;$9mqo|a+vfL!c=ph%hPTYY45 z%s>R!NRQE;105OI8kh9&rG}SI1YKRe2i4-D)-)m#n=bB>kdA;$4S5@8%c`3%Uvu;41`z-E(n_suu3p4 z#*q09A~c+m(j_+8iwAEjo&*{Do*-lH^5wzv066TwLOEg$kOhORV7Qo}0LYQOUD8j* z6ReOKsP!OhjGA!az$RB;VCg<}F`JxI=?&A}jYx~*)OZN5o;spzRYqhwuPA_8+FD0eu zJ_0%4)mIP2jqF)P$jy;Xgk_w(5mt|d{kl8bR*%bykI}x{ZBq^mCE_EKo$--@(LspE z2NyTEde#&Mj0%anWnlEd7M2RFfh&M7Z?!NJbPmTSCSu3p69*^O*G1R7(O4+MVuftj z^RSB%?TJL9y_4WTMj{!HWBT6r#K|Gut@TL00FfNVz9415w}@Yd$JW(_)|M%a_JpIo z;mBQTOs<77U)ZYe`&lkJ(90Z0Vv_^MZ*hr3-$f%E6mYFiAA$P%9EOrbS2*47+TL7< zQSX4ENH^GClK+?OPA5j-8q2BJkiI{5U`^dX;1H}kA$Y=1BV!y!n zKIMC>Z``-K<*O~f*K)BX()`2bKWiRlzp%XjZgnb%c|Ugo53N4nVOh7w^DC@(H+WqB{q5b}05r9MImHxS z_}MpGUH3Dl>VQ|M9q~aLoL0=1ae{VU7j; zUH*HE`WdRm$1o04jiG;vW~`5ob(k64pL}!;(f{Q?4jg{`Vio;A{m2@sf2;pcQUB9a zjoz^ndB??Q?AV_njWcif=o;|K#+3!&$;M z;&J--wr}&c0yZyTyLQUdyFyQTFLUj^W)k%1V`R{*c0i8Xp7Z<{ zpCvg?$Z4?W7&1=ig0GN~8SaikQ(k=cbr_i4{wUP;)^C3VWKTHiJy7N&q)g#RG*T=S z^+$?@Kz%(`uggG_^bfRRPt}}KV~a9oW1+eeG+NvkiFDEWU{e@er>p%#BwBpX;iUls zYI)`b(!?O|arlGAXmcCU;^-%7w0QB|AWKKHo6)1i*|&UvxE;;lu74!FlXXWP`R1;} zPdvmzN9r2K%nxUt{BRh0lJ_+EHy10=hx4&dKSxHbwE}%OpZy#i&Uyu(`QuJf!TZ0t z^YB9tR;%EJ4|LKBR{6ISE70T7F^2J#RiMYC(>KOS7X0?RsRfUHX4l~-t6K1}-vSGu zr*zATE}|a-!xDAZ^E&l&U13u16I+3Lmp|P8=Y;x)9o}ah-Vc%({rG>S)CCXg-&kz3 zEk;vW8}*}J8l!0j_Zy$T!4fJTeeg|(|L%qpuYkC!b`d1IssN!E9HL7UzZz@({i`3RC^sYVnx$lvgMz`YEKea@y zc=YXWJpB2ekf2v?#m&E50xN*Mo>644AKKX0ogoA4T)2~{f5!nJANF^G8QEjCI0&?f2p|EXlhE+YB13`lnPcM(td)`zWi>W6m8 zr>Nwvoka4H8z8yMzmwScH#SNA&@P#yk~>30^2WPCa;HB;?0niLsUO-UcT>sG4kB52 z79>Od9mLMJ*(CKto226>RC335BKg2)L2`$GJF)Y-AF|m=5ABklppx6S5y{1W1j+6G zZN$#c*(CKtyX3=Ea@$rSng1zBZu4&i$sfGmCaE9VB}b^_)-6O*X#>fv{w*N+{G&EW z{m?FX50%`qnMh_kL2`?KGe|!3{XezYNe^w3?3bwI<{**07zfGC{vb#`{@-kp`k`I& zNh%rKL?tIU{*OC1D`XGP3e;sMr-y8Wn&0n{cp) zXRNkWr-M_+7YYTTVCdAf*V=3EeRl4L^GUzRr+c!yerrG0 z`tQB=In&?k_7|yr%4F4EzYE)^z+~nBpl&}z?UOH0-O|j*?*7(moAE!sGy`9hhSx$f z{>Qxk<23WRfFE(1+lKf0C;+Hzil zvV7e>w44VGYHl{gEcwy3tk;$e^~$p9I9fJ9y&C8FVr-oB=vvOzmi2R$W%|!(Sr2p7 zIDZ?nTGjguZ-%Q@PzW)4{bTGqfEv@BkTS@NS}nMtKmmj&xnl{be^ zm(Ru@|Nr+toq@s|7wFPG+qyu-Pwfr3maSOW{7coKr^?b)hT;mIeE0~)1QvODxZPAe zQiCchUG+!}s{Z8FphUCxf^QkzkS_8ZoD_M4 zix+twkBYp`@FwUB3{a{(2Pjouvi^02pMrkNlOoySNs&yvc#-UNkmj#5{4}hvz;0b7 zJ3y(D6LoOvUR4bYk4x_wm;Mpm*pO4aQR#wj=ySB(*fiyO=|+Woxi>cL>0#55-L+wo zy74%9J`+7LA5YO7M>Y!9Y?y?l!1AyczG?V;PH*feszQiDQs3C{IlZmxv}wq$X8wNU zA1TFu>0ZU}2|o!d1LJ^ejru4>H#5!vxYlS{m9G-d*`oy-K>yz}QL&VR9R+lN(#Hcc zCmVjm#F6dOY`LaIZ^mlN)#W=)HrmKGU6>Nd&Dw%U_2`=0Fg+LGJ%)d?c5Ef3=kjnd ze8CKptF{`hmC07mBiG7JYlmt43Yy)$g@*aaXeN4A4U>VmPW)rpx-Gm;Z-*}%KCjc9 zbkSg9>(r>%iTKZNB!1t8OmwsqfA2=-H^A2npO*>uB~?$j{pV$>&*jUdA=|%}_?!Qc ziH?`z|6vXC$K0&xY3mG^z~@bti>j%9d#Xl5iDjifPg>PF;Won)RdaZvYQ%F@&Btj~ zTMl0|I8ikRC#pucn5y|aeRb6XUot!qHHRmnmb7~xhp!l%Xqtl)O(R^A+4}^1)$qEt zq&Ym1v_##*)Z|GpsmYI}Qq#1INlm|{tT3fTQZs5%QF!&8n3@sZ3;VaXLod;#cPAd` z-e*Q7v!twajL9siKgD-WCSWe|57{~E;r)guv*hq(mWb!Zn2*yjHVZyra1vtxSs>kUt0%;8ClCGFk>xWV8gLmix)I|vtxF~55{L(_1h;Yo%%Jju{R-5Vpv z>=U=OI#}w-*_Fjqg~W@NOO~_Q<-F^V#|PJtoc;A=CVEWCnLmhsuV%S?8!R(?mgAXP zj%h5sE%Ubdj^n3kTPFUgLE=AmIuq?I#sB3X^WOy@G<@Dpsz-Y1eT>8B?W8t6OxrT? z?^{Rwm1i^2P$~ZTyOEEn$>TDq$&c~W?gVaj|-&6fFUs;}CPso8=r z6a3U19MP#+18pW4BsGc#@jLb5qmD>C~)%oWV)# zJ2;8`#N8VY3k^?V-{DE@C+*%$c&EXMayU3q4#LHh!ym3zj%v8h@I*Nro+wA6?u|)J z_KB%E2U<;N^GRcRUy2SCQ}fD^{6q<)?4i_YJndn%l{;%K6smXp#n(-s1rn5Kl`zX`m9CbNqO^YP0s{hMV{i(_u!`aqVLrU73f>E4>2qc2GwP-5@Q$Zc!J7}i7r+xzVO6Fi zzLg!RLuE?hTm6Y%99GJULuQ~l@Q-B_-(BJN12|-f65$R}BAkdyA1d{WS070FkPkNo z@T3KbLO4KC2m-3zu~Oa)`$XCuyTVNY9JWN!2!|*dk+h5V2XMp!MPnVH9Dfpb@qqxI zvP4l?hbStWsEbn{SCucl1j84QcKv(99|UkvO2>(~{_1N5dD~Cklu}Ggai-0f#F<7u zA38O?I+%FP&-YJ@inpUu_;Y>YpVP56?F}~sP_!OcE983!#{+8xeh)$0ICt0#sy8-(QJKOvy7%Kq?Z zT3x_xd|^WLr&%aM{y^?JnkEr?IDp5j2bQ^|9Ys3p(|FukLVpFp9xiPjuRQELRuLVV zUc$rQHjgI{>jOAoJ+RCz)zyUKfn`pA$iW^~?5I#4e(}fh=)@aJcsRYif<%bNh6&Ec z4}P=>Zd1~C+H3? zAKo3nkPQk6Gi9ffRl-dDDW0s%z+~kglC0|j*ky?lW)4xpjEHWs`anw7uJCj46U!4h zad@&DC+^|50~oeMkrRg~a+0)%AA_G+-cX7^*K&BGCd7-Wi9ckmCffqoZHb~L4pG!3 zQ4gm+s?H0$$KhQ!UY6U{?+xz?V5cAEOLDz*@+WVcNF3(fdN(JZTes55{DuCi=nNIj z6i_x8Mjup_MNjxy__1}sT5w&&THrX~%{;F^ZzKmV@@4r$cv=2(DF^$hwCD*x3O}_D zSPB-WSqdBneBxQEZ+Y3lHX6Z-fvV`$k`A`%vY$p~5|vjBC(+qexwbGZV-nT0m_)sz zRVQk7TdPXchBi#pHmvge56cD*Q=;~S?|}h3P8OtGu9vn}WgNRLU(&(1<*FgOZzYYh z4{N{ys-cL8E5+z#8&>j4-3$fmfG4u;>7ws|Cvq`=Sw`K?{%$onxC5`V?78Z@=o))+d!)nVD z`EYn5AH<8vhYuwA=nOvsYb;RI!vTtVB<dI literal 0 HcmV?d00001 From 4e6850754cc552aab9bee8fa487e295e0513ff0d Mon Sep 17 00:00:00 2001 From: Li0k Date: Fri, 16 Aug 2024 17:44:25 +0800 Subject: [PATCH 03/26] fix(common): fix hyper-014 compile (#18068) --- src/common/metrics/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/common/metrics/Cargo.toml b/src/common/metrics/Cargo.toml index 0c32b557cebb..444f69c3fca9 100644 --- a/src/common/metrics/Cargo.toml +++ b/src/common/metrics/Cargo.toml @@ -23,7 +23,7 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] } http = "1" http-02 = { package = "http", version = "0.2" } hyper = { version = "1" } -hyper-014 = { package = "hyper", version = "0.14" } +hyper-014 = { package = "hyper", version = "0.14", features = ["client"] } hyper-util = { version = "0.1", features = ["client-legacy"] } hytra = { workspace = true } itertools = { workspace = true } From f5f09a6c15b1291623a1fccc9638035cf19002c0 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Fri, 16 Aug 2024 17:59:38 +0800 Subject: [PATCH 04/26] feat: enable shuffle for snapshot backfill (#18063) --- src/meta/src/stream/stream_graph/actor.rs | 9 +-- src/meta/src/stream/stream_graph/fragment.rs | 42 +++++++------ .../executor/backfill/snapshot_backfill.rs | 61 ++++++++++--------- 3 files changed, 61 insertions(+), 51 deletions(-) diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index 8847ff1e8c02..22424a98f9ab 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -176,13 +176,14 @@ impl ActorBuilder { }]; let upstream_actor_id = upstreams.actors.as_global_ids(); - let is_arrangement_backfill = - stream_scan.stream_scan_type == StreamScanType::ArrangementBackfill as i32; - if !is_arrangement_backfill { + let is_shuffled_backfill = stream_scan.stream_scan_type + == StreamScanType::ArrangementBackfill as i32 + || stream_scan.stream_scan_type == StreamScanType::SnapshotBackfill as i32; + if !is_shuffled_backfill { assert_eq!(upstream_actor_id.len(), 1); } - let upstream_dispatcher_type = if is_arrangement_backfill { + let upstream_dispatcher_type = if is_shuffled_backfill { // FIXME(kwannoel): Should the upstream dispatcher type depends on the upstream distribution? // If singleton, use `Simple` dispatcher, otherwise use `Hash` dispatcher. DispatcherType::Hash as _ diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 185ee3ff9e65..782156775586 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -36,7 +36,7 @@ use risingwave_pb::stream_plan::stream_fragment_graph::{ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ DispatchStrategy, DispatcherType, FragmentTypeFlag, StreamActor, - StreamFragmentGraph as StreamFragmentGraphProto, StreamNode, StreamScanNode, StreamScanType, + StreamFragmentGraph as StreamFragmentGraphProto, StreamScanNode, StreamScanType, }; use crate::barrier::SnapshotBackfillInfo; @@ -206,26 +206,30 @@ impl BuildingFragment { table_columns } - pub fn has_arrangement_backfill(&self) -> bool { - fn has_arrangement_backfill_node(stream_node: &StreamNode) -> bool { - let is_backfill = if let Some(node) = &stream_node.node_body + pub fn has_shuffled_backfill(&self) -> bool { + let stream_node = match self.inner.node.as_ref() { + Some(node) => node, + _ => return false, + }; + let mut has_shuffled_backfill = false; + let has_shuffled_backfill_mut_ref = &mut has_shuffled_backfill; + visit_stream_node_cont(stream_node, |node| { + let is_shuffled_backfill = if let Some(node) = &node.node_body && let Some(node) = node.as_stream_scan() { node.stream_scan_type == StreamScanType::ArrangementBackfill as i32 + || node.stream_scan_type == StreamScanType::SnapshotBackfill as i32 } else { false }; - is_backfill - || stream_node - .get_input() - .iter() - .any(has_arrangement_backfill_node) - } - let stream_node = match self.inner.node.as_ref() { - Some(node) => node, - _ => return false, - }; - has_arrangement_backfill_node(stream_node) + if is_shuffled_backfill { + *has_shuffled_backfill_mut_ref = true; + false + } else { + true + } + }); + has_shuffled_backfill } } @@ -726,7 +730,7 @@ impl CompleteStreamFragmentGraph { }) = upstream_ctx { for (&id, fragment) in &mut graph.fragments { - let uses_arrangement_backfill = fragment.has_arrangement_backfill(); + let uses_shuffled_backfill = fragment.has_shuffled_backfill(); for (&upstream_table_id, output_columns) in &fragment.upstream_table_columns { let (up_fragment_id, edge) = match ddl_type { DdlType::Table(TableJobType::SharedCdcSource) => { @@ -801,7 +805,7 @@ impl CompleteStreamFragmentGraph { (dist_key_indices, output_indices) }; let dispatch_strategy = mv_on_mv_dispatch_strategy( - uses_arrangement_backfill, + uses_shuffled_backfill, dist_key_indices, output_indices, ); @@ -943,11 +947,11 @@ impl CompleteStreamFragmentGraph { } fn mv_on_mv_dispatch_strategy( - uses_arrangement_backfill: bool, + uses_shuffled_backfill: bool, dist_key_indices: Vec, output_indices: Vec, ) -> DispatchStrategy { - if uses_arrangement_backfill { + if uses_shuffled_backfill { if !dist_key_indices.is_empty() { DispatchStrategy { r#type: DispatcherType::Hash as _, diff --git a/src/stream/src/executor/backfill/snapshot_backfill.rs b/src/stream/src/executor/backfill/snapshot_backfill.rs index 53ae800ac536..35adc33b81c4 100644 --- a/src/stream/src/executor/backfill/snapshot_backfill.rs +++ b/src/stream/src/executor/backfill/snapshot_backfill.rs @@ -146,21 +146,6 @@ impl SnapshotBackfillExecutor { let first_barrier_epoch = first_barrier.epoch; - async fn with_consuming_upstream<'a, 'b, T>( - future: impl Future>, - upstream_buffer: &'a mut UpstreamBuffer<'b>, - ) -> StreamExecutorResult { - select! { - biased; - e = upstream_buffer.consume_upstream() => { - Err(e) - } - result = future => { - result - } - } - } - // Phase 1: consume upstream snapshot { { @@ -185,11 +170,9 @@ impl SnapshotBackfillExecutor { pin_mut!(snapshot_stream); - while let Some(message) = with_consuming_upstream( - snapshot_stream.try_next(), - &mut upstream_buffer, - ) - .await? + while let Some(message) = upstream_buffer + .run_future(snapshot_stream.try_next()) + .await? { if let Message::Chunk(chunk) = &message { consuming_snapshot_row_count.inc_by(chunk.cardinality() as _); @@ -230,17 +213,21 @@ impl SnapshotBackfillExecutor { barrier_epoch = barrier.epoch; debug!(?barrier_epoch, kind = ?barrier.kind, "before consume change log"); - let stream = self - .upstream_table - .batch_iter_log_with_pk_bounds(barrier_epoch.prev, barrier_epoch.prev) - .await?; + // use `upstream_buffer.run_future` to poll upstream concurrently so that we won't have back-pressure + // on the upstream. Otherwise, in `batch_iter_log_with_pk_bounds`, we may wait upstream epoch to be committed, + // and the back-pressure may cause the upstream unable to consume the barrier and then cause deadlock. + let stream = + upstream_buffer + .run_future(self.upstream_table.batch_iter_log_with_pk_bounds( + barrier_epoch.prev, + barrier_epoch.prev, + )) + .await?; let data_types = self.upstream_table.schema().data_types(); let builder = create_builder(None, self.chunk_size, data_types); let stream = read_change_log(stream, builder); pin_mut!(stream); - while let Some(chunk) = - with_consuming_upstream(stream.try_next(), &mut upstream_buffer).await? - { + while let Some(chunk) = upstream_buffer.run_future(stream.try_next()).await? { debug!( ?barrier_epoch, size = chunk.cardinality(), @@ -364,7 +351,7 @@ impl<'a> UpstreamBuffer<'a> { } } - async fn consume_upstream(&mut self) -> StreamExecutorError { + async fn concurrently_consume_upstream(&mut self) -> StreamExecutorError { while !self.is_finished { let result = self.consume_until_next_barrier().await; let barrier = match result { @@ -431,6 +418,24 @@ impl<'a> UpstreamBuffer<'a> { false } } + + /// Run a future while concurrently polling the upstream so that the upstream + /// won't be back-pressured. + async fn run_future>( + &mut self, + future: impl Future>, + ) -> StreamExecutorResult { + select! { + biased; + e = self.concurrently_consume_upstream() => { + Err(e) + } + // this arm won't be starved, because the first arm is always pending unless returning with error + result = future => { + result.map_err(Into::into) + } + } + } } async fn receive_next_barrier( From 31e52d4f66745280097e981eb154042cab46296b Mon Sep 17 00:00:00 2001 From: xxchan Date: Sat, 17 Aug 2024 12:22:52 +0800 Subject: [PATCH 05/26] ci: add cherry-pick 2.0 (#18074) Signed-off-by: xxchan --- .../workflows/cherry-pick-to-release-branch.yml | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/.github/workflows/cherry-pick-to-release-branch.yml b/.github/workflows/cherry-pick-to-release-branch.yml index 18dba108047d..363854209f30 100644 --- a/.github/workflows/cherry-pick-to-release-branch.yml +++ b/.github/workflows/cherry-pick-to-release-branch.yml @@ -38,3 +38,19 @@ jobs: env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + + release_pull_request_2_0: + if: "contains(github.event.pull_request.labels.*.name, 'need-cherry-pick-release-2.0') && github.event.pull_request.merged == true" + runs-on: ubuntu-latest + name: release_pull_request + steps: + - name: checkout + uses: actions/checkout@v1 + - name: Create PR to branch + uses: risingwavelabs/github-action-cherry-pick@master + with: + pr_branch: 'release-2.0' + pr_labels: 'cherry-pick' + pr_body: ${{ format('Cherry picking \#{0} onto branch release-2.0', github.event.number) }} + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} From 83b3647ece55ad04d3b4a565c524f8f415d9a71d Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Sat, 17 Aug 2024 15:24:27 +0800 Subject: [PATCH 06/26] fix(stream): fix functional dependencies for row merge (#18072) --- ..._approx_percentile_merge_stateless_agg.slt | 13 ++- .../tests/testdata/input/agg.yaml | 9 ++- .../tests/testdata/output/agg.yaml | 79 +++++++++++++------ .../stream_global_approx_percentile.rs | 6 +- .../stream_local_approx_percentile.rs | 4 +- .../optimizer/plan_node/stream_row_merge.rs | 5 +- 6 files changed, 80 insertions(+), 36 deletions(-) diff --git a/e2e_test/streaming/aggregate/two_phase_approx_percentile_merge_stateless_agg.slt b/e2e_test/streaming/aggregate/two_phase_approx_percentile_merge_stateless_agg.slt index fd54f7200f95..49d5d781f79f 100644 --- a/e2e_test/streaming/aggregate/two_phase_approx_percentile_merge_stateless_agg.slt +++ b/e2e_test/streaming/aggregate/two_phase_approx_percentile_merge_stateless_agg.slt @@ -23,16 +23,15 @@ statement ok create materialized view m1 as select approx_percentile(0.01, 0.01) within group (order by p_col) as p01, - sum(p_col), + sum(p_col) as s, approx_percentile(0.5, 0.01) within group (order by p_col) as p50, - count(*), - approx_percentile(0.99, 0.01) within group (order by p_col) as p99 + count(*)::double + approx_percentile(0.99, 0.01) within group (order by p_col) as p99 from t; query I -select * from m1; +select p01, s, p50, round(p99::numeric, 2) from m1; ---- --982.5779489474152 0 0 2001 982.5779489474152 +-982.5779489474152 0 0 2983.58 # Test state encode / decode onlyif can-use-recover @@ -45,7 +44,7 @@ sleep 10s query I select * from m1; ---- --982.5779489474152 0 0 2001 982.5779489474152 +-982.5779489474152 0 0 2983.5779489474152 # Test 0 Date: Sat, 17 Aug 2024 16:04:52 +0800 Subject: [PATCH 07/26] chore: update Cargo package version to 2.1.0-alpha (#18075) --- Cargo.lock | 116 ++++++++++++++++++++++++++--------------------------- Cargo.toml | 2 +- 2 files changed, 59 insertions(+), 59 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 941d45551075..2ea85047e8b2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3789,7 +3789,7 @@ dependencies = [ [[package]] name = "delta_btree_map" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "educe", "enum-as-inner 0.6.0", @@ -6679,7 +6679,7 @@ checksum = "c4cd1a83af159aa67994778be9070f0ae1bd732942279cabb14f86f986a21456" [[package]] name = "local_stats_alloc" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "workspace-hack", ] @@ -8386,7 +8386,7 @@ dependencies = [ [[package]] name = "pgwire" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "auto_enums", @@ -9818,7 +9818,7 @@ dependencies = [ [[package]] name = "risedev" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "chrono", @@ -9848,7 +9848,7 @@ dependencies = [ [[package]] name = "risedev-config" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "clap", @@ -9861,7 +9861,7 @@ dependencies = [ [[package]] name = "risingwave-fields-derive" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "expect-test", "indoc", @@ -9873,7 +9873,7 @@ dependencies = [ [[package]] name = "risingwave_backup" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "async-trait", @@ -9895,7 +9895,7 @@ dependencies = [ [[package]] name = "risingwave_batch" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "arrow-array 50.0.0", @@ -9953,7 +9953,7 @@ dependencies = [ [[package]] name = "risingwave_bench" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "async-trait", @@ -9995,7 +9995,7 @@ dependencies = [ [[package]] name = "risingwave_cmd" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "clap", "madsim-tokio", @@ -10015,7 +10015,7 @@ dependencies = [ [[package]] name = "risingwave_cmd_all" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "clap", @@ -10048,7 +10048,7 @@ dependencies = [ [[package]] name = "risingwave_common" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "ahash 0.8.11", "anyhow", @@ -10168,7 +10168,7 @@ dependencies = [ [[package]] name = "risingwave_common_estimate_size" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "bytes", "educe", @@ -10183,7 +10183,7 @@ dependencies = [ [[package]] name = "risingwave_common_heap_profiling" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "chrono", @@ -10198,7 +10198,7 @@ dependencies = [ [[package]] name = "risingwave_common_metrics" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "auto_impl", "bytes", @@ -10234,7 +10234,7 @@ dependencies = [ [[package]] name = "risingwave_common_proc_macro" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "bae", "itertools 0.12.1", @@ -10246,7 +10246,7 @@ dependencies = [ [[package]] name = "risingwave_common_secret" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "aes-gcm", "anyhow", @@ -10262,7 +10262,7 @@ dependencies = [ [[package]] name = "risingwave_common_service" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "async-trait", "axum 0.7.4", @@ -10284,7 +10284,7 @@ dependencies = [ [[package]] name = "risingwave_compaction_test" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "async-trait", @@ -10312,7 +10312,7 @@ dependencies = [ [[package]] name = "risingwave_compactor" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "async-trait", "await-tree", @@ -10336,7 +10336,7 @@ dependencies = [ [[package]] name = "risingwave_compute" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "async-trait", @@ -10383,7 +10383,7 @@ dependencies = [ [[package]] name = "risingwave_connector" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "apache-avro 0.16.0", @@ -10521,7 +10521,7 @@ dependencies = [ [[package]] name = "risingwave_connector_codec" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "apache-avro 0.16.0", @@ -10546,7 +10546,7 @@ dependencies = [ [[package]] name = "risingwave_ctl" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "bytes", @@ -10589,7 +10589,7 @@ dependencies = [ [[package]] name = "risingwave_dml" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "assert_matches", "criterion", @@ -10613,7 +10613,7 @@ dependencies = [ [[package]] name = "risingwave_e2e_extended_mode_test" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "chrono", @@ -10628,7 +10628,7 @@ dependencies = [ [[package]] name = "risingwave_error" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "bincode 1.3.3", @@ -10644,7 +10644,7 @@ dependencies = [ [[package]] name = "risingwave_expr" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "arrow-array 50.0.0", @@ -10685,7 +10685,7 @@ dependencies = [ [[package]] name = "risingwave_expr_impl" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "aho-corasick", "anyhow", @@ -10753,7 +10753,7 @@ dependencies = [ [[package]] name = "risingwave_frontend" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "arc-swap", @@ -10838,7 +10838,7 @@ dependencies = [ [[package]] name = "risingwave_frontend_macro" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "proc-macro2", "quote", @@ -10847,7 +10847,7 @@ dependencies = [ [[package]] name = "risingwave_hummock_sdk" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "bytes", "easy-ext", @@ -10866,7 +10866,7 @@ dependencies = [ [[package]] name = "risingwave_hummock_test" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "async-trait", "bytes", @@ -10899,7 +10899,7 @@ dependencies = [ [[package]] name = "risingwave_hummock_trace" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "async-trait", "bincode 2.0.0-rc.3", @@ -10976,7 +10976,7 @@ dependencies = [ [[package]] name = "risingwave_license" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "expect-test", "jsonwebtoken", @@ -10988,7 +10988,7 @@ dependencies = [ [[package]] name = "risingwave_mem_table_spill_test" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "async-trait", "bytes", @@ -11004,7 +11004,7 @@ dependencies = [ [[package]] name = "risingwave_meta" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "arc-swap", @@ -11081,7 +11081,7 @@ dependencies = [ [[package]] name = "risingwave_meta_dashboard" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "axum 0.7.4", @@ -11102,7 +11102,7 @@ dependencies = [ [[package]] name = "risingwave_meta_model_migration" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "async-std", "sea-orm", @@ -11114,7 +11114,7 @@ dependencies = [ [[package]] name = "risingwave_meta_model_v2" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "prost 0.13.1", "risingwave_common", @@ -11127,7 +11127,7 @@ dependencies = [ [[package]] name = "risingwave_meta_node" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "clap", @@ -11161,7 +11161,7 @@ dependencies = [ [[package]] name = "risingwave_meta_service" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "async-trait", @@ -11190,7 +11190,7 @@ dependencies = [ [[package]] name = "risingwave_object_store" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "async-trait", "await-tree", @@ -11227,7 +11227,7 @@ dependencies = [ [[package]] name = "risingwave_pb" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "enum-as-inner 0.6.0", "fs-err", @@ -11248,7 +11248,7 @@ dependencies = [ [[package]] name = "risingwave_planner_test" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "expect-test", @@ -11270,7 +11270,7 @@ dependencies = [ [[package]] name = "risingwave_regress_test" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "clap", @@ -11284,7 +11284,7 @@ dependencies = [ [[package]] name = "risingwave_rpc_client" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "async-trait", @@ -11318,7 +11318,7 @@ dependencies = [ [[package]] name = "risingwave_rt" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "await-tree", "console", @@ -11402,7 +11402,7 @@ dependencies = [ [[package]] name = "risingwave_sqlparser" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "console", @@ -11423,7 +11423,7 @@ dependencies = [ [[package]] name = "risingwave_sqlsmith" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "chrono", @@ -11451,7 +11451,7 @@ dependencies = [ [[package]] name = "risingwave_state_cleaning_test" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "clap", @@ -11471,7 +11471,7 @@ dependencies = [ [[package]] name = "risingwave_storage" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "ahash 0.8.11", "anyhow", @@ -11543,7 +11543,7 @@ dependencies = [ [[package]] name = "risingwave_stream" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "anyhow", "assert_matches", @@ -11612,7 +11612,7 @@ dependencies = [ [[package]] name = "risingwave_test_runner" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "fail", "sync-point", @@ -11621,7 +11621,7 @@ dependencies = [ [[package]] name = "risingwave_variables" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "chrono", "workspace-hack", @@ -15817,7 +15817,7 @@ dependencies = [ [[package]] name = "with_options" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "proc-macro2", "quote", @@ -15838,7 +15838,7 @@ dependencies = [ [[package]] name = "workspace-config" -version = "1.11.0-alpha" +version = "2.1.0-alpha" dependencies = [ "libz-sys", "log", @@ -15850,7 +15850,7 @@ dependencies = [ [[package]] name = "workspace-hack" -version = "1.11.0-alpha" +version = "2.1.0-alpha" [[package]] name = "wyz" diff --git a/Cargo.toml b/Cargo.toml index 9332a55f9041..22ee9c6a1d01 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -69,7 +69,7 @@ exclude = ["e2e_test/udf/wasm", "lints"] resolver = "2" [workspace.package] -version = "1.11.0-alpha" +version = "2.1.0-alpha" edition = "2021" homepage = "https://github.com/risingwavelabs/risingwave" keywords = ["sql", "database", "streaming"] From 7cb75e565470570fd191fac062e91864d4582c98 Mon Sep 17 00:00:00 2001 From: xxchan Date: Sat, 17 Aug 2024 16:41:52 +0800 Subject: [PATCH 08/26] feat: support constructor `MAP {..}` and index operator `m[k]` for map (#18066) Signed-off-by: xxchan --- e2e_test/batch/types/map.slt.part | 55 ++++++++++++-- src/common/src/array/map_array.rs | 2 +- src/expr/impl/src/scalar/array.rs | 4 +- src/frontend/src/binder/expr/mod.rs | 31 ++++++-- src/frontend/src/binder/expr/value.rs | 103 ++++++++++++++++++++++---- src/frontend/src/binder/select.rs | 2 +- src/meta/src/controller/rename.rs | 8 +- src/sqlparser/src/ast/mod.rs | 23 ++++-- src/sqlparser/src/keywords.rs | 1 + src/sqlparser/src/parser.rs | 19 ++++- 10 files changed, 206 insertions(+), 42 deletions(-) diff --git a/e2e_test/batch/types/map.slt.part b/e2e_test/batch/types/map.slt.part index bcdc92103e93..5f68bcad2274 100644 --- a/e2e_test/batch/types/map.slt.part +++ b/e2e_test/batch/types/map.slt.part @@ -29,14 +29,14 @@ db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map('{1,1,3}', '{1,2,3}')` + 2: error while evaluating expression `map_from_entries('{1,1,3}', '{1,2,3}')` 3: map keys must be unique query ? select map_from_entries(array[1,2,3], array[1,null,3]); ---- -{"1":1,"2":NULL,"3":3} +{1:1,2:NULL,3:3} query error @@ -46,7 +46,7 @@ db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map('{1,NULL,3}', '{1,2,3}')` + 2: error while evaluating expression `map_from_entries('{1,NULL,3}', '{1,2,3}')` 3: map keys must not be NULL @@ -57,7 +57,7 @@ db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map('{1,3}', '{1,2,3}')` + 2: error while evaluating expression `map_from_entries('{1,3}', '{1,2,3}')` 3: map keys and values have different length @@ -104,7 +104,7 @@ insert into t values ( query ? select map_from_entries(array['a','b','c'], array[1,2,3])::map(varchar,float); ---- -{"a":1,"b":2,"c":3} +{a:1,b:2,c:3} statement ok @@ -113,8 +113,8 @@ insert into t(m1) values (map_from_entries(array['a','b','c'], array[1,2,3])); query ????? rowsort select * from t; ---- -{"a":1,"b":2,"c":3} NULL NULL NULL NULL -{"a":1,"b":2,"c":3} {"1":t,"2":f,"3":t} {"a":{"a1":a2},"b":{"b1":b2}} {"{\"a\":1,\"b\":2,\"c\":3}","{\"d\":4,\"e\":5,\"f\":6}"} ("{""a"":(1),""b"":(2),""c"":(3)}") +{a:1,b:2,c:3} NULL NULL NULL NULL +{a:1,b:2,c:3} {1:t,2:f,3:t} {a:{a1:a2},b:{b1:b2}} {"{a:1,b:2,c:3}","{d:4,e:5,f:6}"} ("{a:(1),b:(2),c:(3)}") query ????? rowsort select to_jsonb(m1), to_jsonb(m2), to_jsonb(m3), to_jsonb(l), to_jsonb(s) from t; @@ -124,3 +124,44 @@ select to_jsonb(m1), to_jsonb(m2), to_jsonb(m3), to_jsonb(l), to_jsonb(s) from t statement ok drop table t; + +query ?????? +select + MAP {'a':1, 'b':2}, + MAP {'a':1, 'b':2}['b'], + MAP {'a':1, 'b':2}['c'], -- non-existent key + MAP {'a':1, 'b':2}[1], -- type mismatch here + MAP {1:1, '2':2.0}[2], -- implicit cast + MAP {1:MAP{2:3}}[1][2]; +---- +{a:1,b:2} 2 NULL NULL 2.0 3 + + +query error +select MAP {'a':1,'a':2}; +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Expr error + 2: error while evaluating expression `map_from_entries('{a,a}', '{1,2}')` + 3: map keys must be unique + + +query error +select MAP {}; +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: MAP {} + 2: Bind error: cannot determine type of empty map +HINT: Explicitly cast to the desired type, for example MAP{}::map(int,int). + + +query ?? +select + MAP{}::MAP(VARCHAR,INT), + MAP{1:'a',2:'b'}::MAP(VARCHAR,VARCHAR) +---- +{} {1:a,2:b} diff --git a/src/common/src/array/map_array.rs b/src/common/src/array/map_array.rs index 6e9c819a1463..f0904211f5ed 100644 --- a/src/common/src/array/map_array.rs +++ b/src/common/src/array/map_array.rs @@ -431,7 +431,7 @@ impl ToText for MapRef<'_> { let key = key.to_text(); let value = value.to_text(); // TODO: consider quote like list and struct - f(&format_args!("\"{}\":{}", key, value)) + f(&format_args!("{}:{}", key, value)) }) ) } diff --git a/src/expr/impl/src/scalar/array.rs b/src/expr/impl/src/scalar/array.rs index 08de9714ce05..cee7de36c717 100644 --- a/src/expr/impl/src/scalar/array.rs +++ b/src/expr/impl/src/scalar/array.rs @@ -48,13 +48,13 @@ fn map_type_infer(args: &[DataType]) -> Result { /// query T /// select map_from_entries(array['a','b','c'], array[1,2,3]); /// ---- -/// {"a":1,"b":2,"c":3} +/// {a:1,b:2,c:3} /// ``` #[function( "map_from_entries(anyarray, anyarray) -> anymap", type_infer = "map_type_infer" )] -fn map(key: ListRef<'_>, value: ListRef<'_>) -> Result { +fn map_from_entries(key: ListRef<'_>, value: ListRef<'_>) -> Result { MapValue::try_from_kv(key.to_owned(), value.to_owned()).map_err(ExprError::Custom) } diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index d11134b097e3..3c127c7da7c4 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -102,7 +102,7 @@ impl Binder { Expr::BinaryOp { left, op, right } => self.bind_binary_op(*left, op, *right), Expr::Nested(expr) => self.bind_expr_inner(*expr), Expr::Array(Array { elem: exprs, .. }) => self.bind_array(exprs), - Expr::ArrayIndex { obj, index } => self.bind_array_index(*obj, *index), + Expr::Index { obj, index } => self.bind_index(*obj, *index), Expr::ArrayRangeIndex { obj, start, end } => { self.bind_array_range_index(*obj, start, end) } @@ -194,7 +194,19 @@ impl Binder { Expr::Parameter { index } => self.bind_parameter(index), Expr::Collate { expr, collation } => self.bind_collate(*expr, collation), Expr::ArraySubquery(q) => self.bind_subquery_expr(*q, SubqueryKind::Array), - _ => bail_not_implemented!(issue = 112, "unsupported expression {:?}", expr), + Expr::Map { entries } => self.bind_map(entries), + Expr::IsJson { + unique_keys: true, .. + } + | Expr::SomeOp(_) + | Expr::AllOp(_) + | Expr::TryCast { .. } + | Expr::GroupingSets(_) + | Expr::Cube(_) + | Expr::Rollup(_) + | Expr::LambdaFunction { .. } => { + bail_not_implemented!(issue = 112, "unsupported expression {:?}", expr) + } } } @@ -910,13 +922,16 @@ impl Binder { } pub fn bind_cast_inner(&mut self, expr: Expr, data_type: DataType) -> Result { - if let Expr::Array(Array { elem: ref expr, .. }) = expr - && matches!(&data_type, DataType::List { .. }) - { - return self.bind_array_cast(expr.clone(), data_type); + match (expr, data_type) { + (Expr::Array(Array { elem: ref expr, .. }), DataType::List(element_type)) => { + self.bind_array_cast(expr.clone(), element_type) + } + (Expr::Map { entries }, DataType::Map(m)) => self.bind_map_cast(entries, m), + (expr, data_type) => { + let lhs = self.bind_expr_inner(expr)?; + lhs.cast_explicit(data_type).map_err(Into::into) + } } - let lhs = self.bind_expr_inner(expr)?; - lhs.cast_explicit(data_type).map_err(Into::into) } pub fn bind_collate(&mut self, expr: Expr, collation: ObjectName) -> Result { diff --git a/src/frontend/src/binder/expr/value.rs b/src/frontend/src/binder/expr/value.rs index 5b69610f13bf..711aa6bbb697 100644 --- a/src/frontend/src/binder/expr/value.rs +++ b/src/frontend/src/binder/expr/value.rs @@ -14,7 +14,7 @@ use itertools::Itertools; use risingwave_common::bail_not_implemented; -use risingwave_common::types::{DataType, DateTimeField, Decimal, Interval, ScalarImpl}; +use risingwave_common::types::{DataType, DateTimeField, Decimal, Interval, MapType, ScalarImpl}; use risingwave_sqlparser::ast::{DateTimeField as AstDateTimeField, Expr, Value}; use thiserror_ext::AsReport; @@ -132,27 +132,92 @@ impl Binder { Ok(expr) } - pub(super) fn bind_array_cast(&mut self, exprs: Vec, ty: DataType) -> Result { - let inner_type = if let DataType::List(datatype) = &ty { - *datatype.clone() - } else { - return Err(ErrorCode::BindError(format!( - "cannot cast array to non-array type {}", - ty - )) - .into()); - }; + pub(super) fn bind_map(&mut self, entries: Vec<(Expr, Expr)>) -> Result { + if entries.is_empty() { + return Err(ErrorCode::BindError("cannot determine type of empty map\nHINT: Explicitly cast to the desired type, for example MAP{}::map(int,int).".into()).into()); + } + let mut keys = Vec::with_capacity(entries.len()); + let mut values = Vec::with_capacity(entries.len()); + for (k, v) in entries { + keys.push(self.bind_expr_inner(k)?); + values.push(self.bind_expr_inner(v)?); + } + let key_type = align_types(keys.iter_mut())?; + let value_type = align_types(values.iter_mut())?; + + let keys: ExprImpl = FunctionCall::new_unchecked( + ExprType::Array, + keys, + DataType::List(Box::new(key_type.clone())), + ) + .into(); + let values: ExprImpl = FunctionCall::new_unchecked( + ExprType::Array, + values, + DataType::List(Box::new(value_type.clone())), + ) + .into(); + + let expr: ExprImpl = FunctionCall::new_unchecked( + ExprType::MapFromEntries, + vec![keys, values], + DataType::Map(MapType::from_kv(key_type, value_type)), + ) + .into(); + Ok(expr) + } + pub(super) fn bind_array_cast( + &mut self, + exprs: Vec, + element_type: Box, + ) -> Result { let exprs = exprs .into_iter() - .map(|e| self.bind_cast_inner(e, inner_type.clone())) + .map(|e| self.bind_cast_inner(e, *element_type.clone())) .collect::>>()?; - let expr: ExprImpl = FunctionCall::new_unchecked(ExprType::Array, exprs, ty).into(); + let expr: ExprImpl = + FunctionCall::new_unchecked(ExprType::Array, exprs, DataType::List(element_type)) + .into(); Ok(expr) } - pub(super) fn bind_array_index(&mut self, obj: Expr, index: Expr) -> Result { + pub(super) fn bind_map_cast( + &mut self, + entries: Vec<(Expr, Expr)>, + map_type: MapType, + ) -> Result { + let mut keys = Vec::with_capacity(entries.len()); + let mut values = Vec::with_capacity(entries.len()); + for (k, v) in entries { + keys.push(self.bind_cast_inner(k, map_type.key().clone())?); + values.push(self.bind_cast_inner(v, map_type.value().clone())?); + } + + let keys: ExprImpl = FunctionCall::new_unchecked( + ExprType::Array, + keys, + DataType::List(Box::new(map_type.key().clone())), + ) + .into(); + let values: ExprImpl = FunctionCall::new_unchecked( + ExprType::Array, + values, + DataType::List(Box::new(map_type.value().clone())), + ) + .into(); + + let expr: ExprImpl = FunctionCall::new_unchecked( + ExprType::MapFromEntries, + vec![keys, values], + DataType::Map(map_type), + ) + .into(); + Ok(expr) + } + + pub(super) fn bind_index(&mut self, obj: Expr, index: Expr) -> Result { let obj = self.bind_expr_inner(obj)?; match obj.return_type() { DataType::List(return_type) => Ok(FunctionCall::new_unchecked( @@ -161,8 +226,14 @@ impl Binder { *return_type, ) .into()), + DataType::Map(m) => Ok(FunctionCall::new_unchecked( + ExprType::MapAccess, + vec![obj, self.bind_expr_inner(index)?], + m.value().clone(), + ) + .into()), data_type => Err(ErrorCode::BindError(format!( - "array index applied to type {}, which is not a composite type", + "index operator applied to type {}, which is not a list or map", data_type )) .into()), @@ -198,7 +269,7 @@ impl Binder { ) .into()), data_type => Err(ErrorCode::BindError(format!( - "array range index applied to type {}, which is not a composite type", + "array range index applied to type {}, which is not a list", data_type )) .into()), diff --git a/src/frontend/src/binder/select.rs b/src/frontend/src/binder/select.rs index e6eead2bc08d..0f855f186de4 100644 --- a/src/frontend/src/binder/select.rs +++ b/src/frontend/src/binder/select.rs @@ -683,7 +683,7 @@ fn derive_alias(expr: &Expr) -> Option { Expr::Value(Value::Interval { .. }) => Some("interval".to_string()), Expr::Row(_) => Some("row".to_string()), Expr::Array(_) => Some("array".to_string()), - Expr::ArrayIndex { obj, index: _ } => derive_alias(&obj), + Expr::Index { obj, index: _ } => derive_alias(&obj), _ => None, } } diff --git a/src/meta/src/controller/rename.rs b/src/meta/src/controller/rename.rs index 86465e286d95..5f10a8946414 100644 --- a/src/meta/src/controller/rename.rs +++ b/src/meta/src/controller/rename.rs @@ -313,7 +313,7 @@ impl QueryRewriter<'_> { | Expr::Overlay { expr, .. } | Expr::Trim { expr, .. } | Expr::Nested(expr) - | Expr::ArrayIndex { obj: expr, .. } + | Expr::Index { obj: expr, .. } | Expr::ArrayRangeIndex { obj: expr, .. } => self.visit_expr(expr), Expr::Position { substring, string } => { @@ -379,6 +379,12 @@ impl QueryRewriter<'_> { self.visit_expr(expr); } } + Expr::Map { entries } => { + for (key, value) in entries { + self.visit_expr(key); + self.visit_expr(value); + } + } Expr::LambdaFunction { body, args: _ } => self.visit_expr(body), diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index b0cd783a637c..73c174769eb4 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -502,8 +502,8 @@ pub enum Expr { Array(Array), /// An array constructing subquery `ARRAY(SELECT 2 UNION SELECT 3)` ArraySubquery(Box), - /// A subscript expression `arr[1]` - ArrayIndex { + /// A subscript expression `arr[1]` or `map['a']` + Index { obj: Box, index: Box, }, @@ -517,6 +517,9 @@ pub enum Expr { args: Vec, body: Box, }, + Map { + entries: Vec<(Expr, Expr)>, + }, } impl fmt::Display for Expr { @@ -797,7 +800,7 @@ impl fmt::Display for Expr { .as_slice() .join(", ") ), - Expr::ArrayIndex { obj, index } => { + Expr::Index { obj, index } => { write!(f, "{}[{}]", obj, index)?; Ok(()) } @@ -823,6 +826,16 @@ impl fmt::Display for Expr { body ) } + Expr::Map { entries } => { + write!( + f, + "MAP {{{}}}", + entries + .iter() + .map(|(k, v)| format!("{}: {}", k, v)) + .join(", ") + ) + } } } } @@ -3349,13 +3362,13 @@ mod tests { #[test] fn test_array_index_display() { - let array_index = Expr::ArrayIndex { + let array_index = Expr::Index { obj: Box::new(Expr::Identifier(Ident::new_unchecked("v1"))), index: Box::new(Expr::Value(Value::Number("1".into()))), }; assert_eq!("v1[1]", format!("{}", array_index)); - let array_index2 = Expr::ArrayIndex { + let array_index2 = Expr::Index { obj: Box::new(array_index), index: Box::new(Expr::Value(Value::Number("1".into()))), }; diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index 768301544ef2..79036ea849a3 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -313,6 +313,7 @@ define_keywords!( LOGICAL, LOGIN, LOWER, + MAP, MATCH, MATERIALIZED, MAX, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index b0473f280bf2..e239bfb3b0a6 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -608,6 +608,7 @@ impl Parser<'_> { Ok(exists_node) } Keyword::ARRAY if self.peek_token() == Token::LBracket => self.parse_array_expr(), + Keyword::MAP if self.peek_token() == Token::LBrace => self.parse_map_expr(), // `LEFT` and `RIGHT` are reserved as identifier but okay as function Keyword::LEFT | Keyword::RIGHT => { *self = checkpoint; @@ -1149,6 +1150,22 @@ impl Parser<'_> { Ok(exprs) } + /// Parses a map expression `MAP {k1:v1, k2:v2, ..}` + pub fn parse_map_expr(&mut self) -> PResult { + self.expect_token(&Token::LBrace)?; + if self.consume_token(&Token::RBrace) { + return Ok(Expr::Map { entries: vec![] }); + } + let entries = self.parse_comma_separated(|parser| { + let key = parser.parse_expr()?; + parser.expect_token(&Token::Colon)?; + let value = parser.parse_expr()?; + Ok((key, value)) + })?; + self.expect_token(&Token::RBrace)?; + Ok(Expr::Map { entries }) + } + // This function parses date/time fields for interval qualifiers. pub fn parse_date_time_field(&mut self) -> PResult { dispatch! { peek(keyword); @@ -1554,7 +1571,7 @@ impl Parser<'_> { } _ => { // [N] - Expr::ArrayIndex { + Expr::Index { obj: Box::new(expr), index, } From e36a630dfe0ee60ac9b4293c76f87fc2f035a59b Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Mon, 19 Aug 2024 11:49:48 +0800 Subject: [PATCH 09/26] chore: fix RustRover macro expansion (#18077) --- src/common/src/array/iterator.rs | 6 +-- src/common/src/array/mod.rs | 14 +++--- src/common/src/test_utils/rand_array.rs | 6 +-- src/common/src/types/macros.rs | 57 ------------------------- src/common/src/types/mod.rs | 19 ++++----- src/common/src/util/schema_check.rs | 8 ++-- src/expr/core/src/expr/value.rs | 6 +-- 7 files changed, 29 insertions(+), 87 deletions(-) diff --git a/src/common/src/array/iterator.rs b/src/common/src/array/iterator.rs index 78c89f422aa0..31518150e6f8 100644 --- a/src/common/src/array/iterator.rs +++ b/src/common/src/array/iterator.rs @@ -56,10 +56,10 @@ mod tests { use super::*; use crate::array::{ArrayBuilder, ArrayImpl}; - use crate::for_all_array_variants; + use crate::for_all_variants; macro_rules! test_trusted_len { - ($( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { $( paste! { #[test] @@ -91,5 +91,5 @@ mod tests { }; } - for_all_array_variants! { test_trusted_len } + for_all_variants! { test_trusted_len } } diff --git a/src/common/src/array/mod.rs b/src/common/src/array/mod.rs index ae6f7d0fa144..b34e5f9b9c47 100644 --- a/src/common/src/array/mod.rs +++ b/src/common/src/array/mod.rs @@ -68,7 +68,7 @@ pub use self::error::ArrayError; pub use crate::array::num256_array::{Int256Array, Int256ArrayBuilder}; use crate::bitmap::Bitmap; use crate::types::*; -use crate::{dispatch_array_builder_variants, dispatch_array_variants, for_all_array_variants}; +use crate::{dispatch_array_builder_variants, dispatch_array_variants, for_all_variants}; pub type ArrayResult = Result; pub type I64Array = PrimitiveArray; @@ -325,7 +325,7 @@ impl CompactableArray for A { /// Define `ArrayImpl` with macro. macro_rules! array_impl_enum { - ( $( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ( $( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { /// `ArrayImpl` embeds all possible array in `array` module. #[derive(Debug, Clone, EstimateSize)] pub enum ArrayImpl { @@ -334,7 +334,7 @@ macro_rules! array_impl_enum { }; } -for_all_array_variants! { array_impl_enum } +for_all_variants! { array_impl_enum } // We cannot put the From implementations in impl_convert, // because then we can't prove for all `T: PrimitiveArrayItemType`, @@ -401,7 +401,7 @@ impl From for ArrayImpl { /// * `ArrayImpl -> Array` with `From` trait. /// * `ArrayBuilder -> ArrayBuilderImpl` with `From` trait. macro_rules! impl_convert { - ($( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { $( paste! { impl ArrayImpl { @@ -455,11 +455,11 @@ macro_rules! impl_convert { }; } -for_all_array_variants! { impl_convert } +for_all_variants! { impl_convert } /// Define `ArrayImplBuilder` with macro. macro_rules! array_builder_impl_enum { - ($( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { /// `ArrayBuilderImpl` embeds all possible array in `array` module. #[derive(Debug, Clone, EstimateSize)] pub enum ArrayBuilderImpl { @@ -468,7 +468,7 @@ macro_rules! array_builder_impl_enum { }; } -for_all_array_variants! { array_builder_impl_enum } +for_all_variants! { array_builder_impl_enum } /// Implements all `ArrayBuilder` functions with `for_all_variant`. impl ArrayBuilderImpl { diff --git a/src/common/src/test_utils/rand_array.rs b/src/common/src/test_utils/rand_array.rs index a7c13e3178f2..f201b4de3384 100644 --- a/src/common/src/test_utils/rand_array.rs +++ b/src/common/src/test_utils/rand_array.rs @@ -201,12 +201,12 @@ where #[cfg(test)] mod tests { use super::*; - use crate::for_all_array_variants; + use crate::for_all_variants; #[test] fn test_create_array() { macro_rules! gen_rand_array { - ($( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { $( { let array = seed_rand_array::<$array>(10, 1024, 0.5); @@ -216,6 +216,6 @@ mod tests { }; } - for_all_array_variants! { gen_rand_array } + for_all_variants! { gen_rand_array } } } diff --git a/src/common/src/types/macros.rs b/src/common/src/types/macros.rs index 1dd29156dd65..9fa9c7ffe5ec 100644 --- a/src/common/src/types/macros.rs +++ b/src/common/src/types/macros.rs @@ -64,63 +64,6 @@ macro_rules! for_all_variants { }; } -/// The projected version of `for_all_variants` for handling scalar variants. -/// -/// Arguments are `$variant_name`, `$suffix_name`, `$scalar`, `$scalar_ref`. -#[macro_export(local_inner_macros)] -macro_rules! for_all_scalar_variants { - ($macro:ident $(, $x:tt)*) => { - for_all_variants! { project_scalar_variants, $macro, [ $($x, )* ] } - }; -} -#[macro_export] -macro_rules! project_scalar_variants { - ($macro:ident, [ $($x:tt, )* ], $( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { - $macro! { - $($x, )* - $( { $variant_name, $suffix_name, $scalar, $scalar_ref } ),* - } - }; -} - -/// The projected version of `for_all_variants` for handling array variants. -/// -/// Arguments are `$variant_name`, `$suffix_name`, `$array`, `$builder`. -#[macro_export(local_inner_macros)] -macro_rules! for_all_array_variants { - ($macro:ident $(, $x:tt)*) => { - for_all_variants! { project_array_variants, $macro, [ $($x, )* ] } - }; -} -#[macro_export] -macro_rules! project_array_variants { - ($macro:ident, [ $($x:tt, )* ], $( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { - $macro! { - $($x, )* - $( { $variant_name, $suffix_name, $array, $builder } ),* - } - }; -} - -/// The projected version of `for_all_variants` for handling mapping of data types and array types. -/// -/// Arguments are `$data_type`, `$variant_name`. -#[macro_export(local_inner_macros)] -macro_rules! for_all_type_pairs { - ($macro:ident $(, $x:tt)*) => { - for_all_variants! { project_type_pairs, $macro, [ $($x, )* ] } - }; -} -#[macro_export] -macro_rules! project_type_pairs { - ($macro:ident, [ $($x:tt, )* ], $( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { - $macro! { - $($x, )* - $( { $data_type, $variant_name } ),* - } - }; -} - /// Helper macro for expanding type aliases and constants. Internally used by `dispatch_` macros. #[macro_export] macro_rules! do_expand_alias { diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index e76bfaba384f..b86e70b85d8b 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -42,8 +42,7 @@ pub use crate::array::{ListRef, ListValue, MapRef, MapValue, StructRef, StructVa use crate::cast::{str_to_bool, str_to_bytea}; use crate::error::BoxedError; use crate::{ - dispatch_data_types, dispatch_scalar_ref_variants, dispatch_scalar_variants, - for_all_scalar_variants, for_all_type_pairs, + dispatch_data_types, dispatch_scalar_ref_variants, dispatch_scalar_variants, for_all_variants, }; mod cow; @@ -552,7 +551,7 @@ pub trait ScalarRef<'a>: private::ScalarBounds> + 'a + Copy { /// Define `ScalarImpl` and `ScalarRefImpl` with macro. macro_rules! scalar_impl_enum { - ($( { $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { /// `ScalarImpl` embeds all possible scalars in the evaluation framework. /// /// Note: `ScalarImpl` doesn't contain all information of its `DataType`, @@ -580,7 +579,7 @@ macro_rules! scalar_impl_enum { }; } -for_all_scalar_variants! { scalar_impl_enum } +for_all_variants! { scalar_impl_enum } // We MUST NOT implement `Ord` for `ScalarImpl` because that will make `Datum` derive an incorrect // default `Ord`. To get a default-ordered `ScalarImpl`/`ScalarRefImpl`/`Datum`/`DatumRef`, you can @@ -686,7 +685,7 @@ macro_rules! for_all_native_types { /// * `&ScalarImpl -> &Scalar` with `impl.as_int16()`. /// * `ScalarImpl -> Scalar` with `impl.into_int16()`. macro_rules! impl_convert { - ($( { $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { $( impl From<$scalar> for ScalarImpl { fn from(val: $scalar) -> Self { @@ -758,7 +757,7 @@ macro_rules! impl_convert { }; } -for_all_scalar_variants! { impl_convert } +for_all_variants! { impl_convert } // Implement `From` for `ScalarImpl::Float` as a sugar. impl From for ScalarImpl { @@ -1092,16 +1091,16 @@ pub fn literal_type_match(data_type: &DataType, literal: Option<&ScalarImpl>) -> match literal { Some(scalar) => { macro_rules! matches { - ($( { $DataType:ident, $PhysicalType:ident }),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty }),*) => { match (data_type, scalar) { $( - (DataType::$DataType { .. }, ScalarImpl::$PhysicalType(_)) => true, - (DataType::$DataType { .. }, _) => false, // so that we won't forget to match a new logical type + (DataType::$data_type { .. }, ScalarImpl::$variant_name(_)) => true, + (DataType::$data_type { .. }, _) => false, // so that we won't forget to match a new logical type )* } } } - for_all_type_pairs! { matches } + for_all_variants! { matches } } None => true, } diff --git a/src/common/src/util/schema_check.rs b/src/common/src/util/schema_check.rs index 850329766577..f035ed713793 100644 --- a/src/common/src/util/schema_check.rs +++ b/src/common/src/util/schema_check.rs @@ -15,7 +15,7 @@ use itertools::Itertools; use crate::array::{ArrayImpl, ArrayRef}; -use crate::for_all_type_pairs; +use crate::for_all_variants; use crate::types::DataType; /// Check if the schema of `columns` matches the expected `data_types`. Used for debugging. @@ -30,9 +30,9 @@ where .enumerate() { macro_rules! matches { - ($( { $DataType:ident, $PhysicalType:ident }),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty }),*) => { match (pair.as_ref().left(), pair.as_ref().right()) { - $( (Some(DataType::$DataType { .. }), Some(ArrayImpl::$PhysicalType(_))) => continue, )* + $( (Some(DataType::$data_type { .. }), Some(ArrayImpl::$variant_name(_))) => continue, )* (data_type, array) => { let array_ident = array.map(|a| a.get_ident()); return Err(format!( @@ -43,7 +43,7 @@ where } } - for_all_type_pairs! { matches } + for_all_variants! { matches } } Ok(()) diff --git a/src/expr/core/src/expr/value.rs b/src/expr/core/src/expr/value.rs index c4ec77268488..00aa2bf71ce1 100644 --- a/src/expr/core/src/expr/value.rs +++ b/src/expr/core/src/expr/value.rs @@ -14,7 +14,7 @@ use either::Either; use risingwave_common::array::*; -use risingwave_common::for_all_array_variants; +use risingwave_common::for_all_variants; use risingwave_common::types::{Datum, DatumRef, Scalar, ToDatumRef}; /// The type-erased return value of an expression. @@ -79,7 +79,7 @@ impl<'a, A: Array> ValueRef<'a, A> { } macro_rules! impl_convert { - ($( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { $( paste::paste! { /// Converts a type-erased value to a reference of a specific array type. @@ -102,4 +102,4 @@ macro_rules! impl_convert { }; } -for_all_array_variants! { impl_convert } +for_all_variants! { impl_convert } From 33dc6fd65c9332e6c2751021652e5766802cf87e Mon Sep 17 00:00:00 2001 From: lmatz Date: Mon, 19 Aug 2024 12:22:40 +0800 Subject: [PATCH 10/26] chore: remind labeling new features or improvements as experiemental if needed (#18084) --- .github/workflows/auto-create-doc-issue-by-pr.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/auto-create-doc-issue-by-pr.yml b/.github/workflows/auto-create-doc-issue-by-pr.yml index 533b5c6c09e4..d5b641de5664 100644 --- a/.github/workflows/auto-create-doc-issue-by-pr.yml +++ b/.github/workflows/auto-create-doc-issue-by-pr.yml @@ -39,7 +39,7 @@ jobs: - name: Create issue in other repository if: steps.check_merged.outputs.merged == 'true' && steps.check_documentation_update.outputs.documentation_update == 'true' run: | - ISSUE_CONTENT="This issue tracks the documentation update needed for the merged PR #$PR_ID.\n\nSource PR URL: $PR_URL\nSource PR Merged At: $PR_MERGED_AT" + ISSUE_CONTENT="This issue tracks the documentation update needed for the merged PR #$PR_ID.\n\nSource PR URL: $PR_URL\nSource PR Merged At: $PR_MERGED_AT\n\nIf it is a major improvement that deserves a new page or a new section in the documentation, please check if we should label it as an experiment feature." curl -X POST \ -H "Authorization: Bearer ${{ secrets.ACCESS_TOKEN }}" \ From d16847d1057990cc1ef4c4283503d39d64a08f75 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Mon, 19 Aug 2024 12:54:39 +0800 Subject: [PATCH 11/26] feat(frontend): bind default value for `approx_percentile` relative_error (#18082) --- .../tests/testdata/input/agg.yaml | 7 ++++ .../tests/testdata/output/agg.yaml | 16 ++++++++ .../src/binder/expr/function/aggregate.rs | 41 ++++++++++++------- 3 files changed, 50 insertions(+), 14 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 1979e4ea1fb7..75aa7249accc 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -1067,6 +1067,13 @@ sql: | CREATE TABLE t (v1 int, v2 int); SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, count(*), max(v2) as m2, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v2) as y from t; + expected_outputs: + - logical_plan + - stream_plan +- name: test approx percentile with default relative_error + sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5) WITHIN GROUP (order by v1) from t; expected_outputs: - logical_plan - stream_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index eca739788bf6..9fd70f1fb28f 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -2103,3 +2103,19 @@ └─StreamHashAgg { group_key: [$expr5], aggs: [sum(t.v1), count, max(t.v2)] } └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr3, t.v2, t.v2::Float64 as $expr4, t._row_id, Vnode(t._row_id) as $expr5] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test approx percentile with default relative_error + sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5) WITHIN GROUP (order by v1) from t; + logical_plan: |- + LogicalProject { exprs: [approx_percentile($expr1)] } + └─LogicalAgg { aggs: [approx_percentile($expr1)] } + └─LogicalProject { exprs: [t.v1::Float64 as $expr1] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamExchange { dist: Single } + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/binder/expr/function/aggregate.rs b/src/frontend/src/binder/expr/function/aggregate.rs index 1e7b76bf7629..d6410616c1d9 100644 --- a/src/frontend/src/binder/expr/function/aggregate.rs +++ b/src/frontend/src/binder/expr/function/aggregate.rs @@ -14,7 +14,7 @@ use itertools::Itertools; use risingwave_common::bail_not_implemented; -use risingwave_common::types::DataType; +use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_expr::aggregate::{agg_kinds, AggKind, PbAggKind}; use risingwave_sqlparser::ast::{Function, FunctionArgExpr}; @@ -139,12 +139,9 @@ impl Binder { let order_by = OrderBy::new(vec![self.bind_order_by_expr(within_group)?]); // check signature and do implicit cast - match (&kind, direct_args.as_mut_slice(), args.as_mut_slice()) { - ( - AggKind::Builtin(PbAggKind::PercentileCont | PbAggKind::PercentileDisc), - [fraction], - [arg], - ) => { + match (&kind, direct_args.len(), args.as_mut_slice()) { + (AggKind::Builtin(PbAggKind::PercentileCont | PbAggKind::PercentileDisc), 1, [arg]) => { + let fraction = &mut direct_args[0]; decimal_to_float64(fraction, &kind)?; if matches!(&kind, AggKind::Builtin(PbAggKind::PercentileCont)) { arg.cast_implicit_mut(DataType::Float64).map_err(|_| { @@ -155,14 +152,30 @@ impl Binder { })?; } } - (AggKind::Builtin(PbAggKind::Mode), [], [_arg]) => {} - ( - AggKind::Builtin(PbAggKind::ApproxPercentile), - [percentile, relative_error], - [_percentile_col], - ) => { + (AggKind::Builtin(PbAggKind::Mode), 0, [_arg]) => {} + (AggKind::Builtin(PbAggKind::ApproxPercentile), 1..=2, [_percentile_col]) => { + let percentile = &mut direct_args[0]; decimal_to_float64(percentile, &kind)?; - decimal_to_float64(relative_error, &kind)?; + match direct_args.len() { + 2 => { + let relative_error = &mut direct_args[1]; + decimal_to_float64(relative_error, &kind)?; + } + 1 => { + let relative_error: ExprImpl = Literal::new( + ScalarImpl::Float64(0.01.into()).into(), + DataType::Float64, + ) + .into(); + direct_args.push(relative_error); + } + _ => { + return Err(ErrorCode::InvalidInputSyntax( + "invalid direct args for approx_percentile aggregation".to_string(), + ) + .into()) + } + } } _ => { return Err(ErrorCode::InvalidInputSyntax(format!( From 1699b3ed093ba7c516f1ce2da7ad7669cd01f8c5 Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 19 Aug 2024 13:47:23 +0800 Subject: [PATCH 12/26] refactor: reduce periodical logs (#18080) Signed-off-by: xxchan --- ci/scripts/e2e-source-test.sh | 2 +- src/meta/src/barrier/mod.rs | 8 ++++---- .../src/compaction_group/hummock_version_ext.rs | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index b6eff317a79e..5cdd4b9e3999 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -130,7 +130,7 @@ echo "> inserted new rows into postgres" # start cluster w/o clean-data unset RISINGWAVE_CI -export RUST_LOG="events::stream::message::chunk=trace,risingwave_stream=debug,risingwave_batch=info,risingwave_storage=info" \ +export RUST_LOG="risingwave_stream=debug,risingwave_batch=info,risingwave_storage=info" \ risedev dev ci-1cn-1fe-with-recovery echo "> wait for cluster recovery finish" diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 7d5c954fde73..e349fadf4837 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -309,7 +309,7 @@ impl CheckpointControl { ); } - debug!( + tracing::trace!( prev_epoch = command_ctx.prev_epoch.value().0, ?jobs_to_wait, "enqueue command" @@ -354,7 +354,7 @@ impl CheckpointControl { fn barrier_collected(&mut self, resp: BarrierCompleteResponse) { let worker_id = resp.worker_id; let prev_epoch = resp.epoch; - debug!( + tracing::trace!( worker_id, prev_epoch, partial_graph_id = resp.partial_graph_id, @@ -1005,7 +1005,7 @@ impl GlobalBarrierManager { BarrierKind::Barrier }; - debug!(prev_epoch = prev_epoch.value().0, "inject barrier"); + tracing::trace!(prev_epoch = prev_epoch.value().0, "inject barrier"); // Collect the jobs to finish if let (BarrierKind::Checkpoint(_), Command::Plain(None)) = (&kind, &command) @@ -1206,7 +1206,7 @@ impl GlobalBarrierManagerContext { mut finished_jobs: Vec, backfill_pinned_log_epoch: HashMap)>, ) -> MetaResult> { - debug!( + tracing::trace!( prev_epoch = node.command_ctx.prev_epoch.value().0, kind = ?node.command_ctx.kind, "complete barrier" diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index f7305fa0a150..13a0bcc08adf 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -565,7 +565,7 @@ impl HummockVersion { && self.visible_table_committed_epoch() < version_delta.visible_table_committed_epoch() { is_commit_epoch = true; - warn!("max committed epoch bumped but no table committed epoch is changed"); + tracing::trace!("max committed epoch bumped but no table committed epoch is changed"); } // apply to `levels`, which is different compaction groups From a10e61f8500de49ca669638db82f52209fbc22f6 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Mon, 19 Aug 2024 14:26:42 +0800 Subject: [PATCH 13/26] test(frontend): test two phase approx percentile with group key is banned (#18085) --- src/frontend/planner_test/tests/testdata/input/agg.yaml | 7 +++++++ src/frontend/planner_test/tests/testdata/output/agg.yaml | 8 ++++++++ src/frontend/src/optimizer/plan_node/logical_agg.rs | 5 ++++- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 75aa7249accc..9310a826a691 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -1070,6 +1070,13 @@ expected_outputs: - logical_plan - stream_plan +- name: test approx_percentile hash_agg forced should use single phase agg + sql: | + SET RW_FORCE_TWO_PHASE_AGG=true; + create table t (v1 int, grp_col int); + select approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t group by grp_col; + expected_outputs: + - stream_error - name: test approx percentile with default relative_error sql: | CREATE TABLE t (v1 int); diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 9fd70f1fb28f..97d440b64df1 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -2103,6 +2103,14 @@ └─StreamHashAgg { group_key: [$expr5], aggs: [sum(t.v1), count, max(t.v2)] } └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr3, t.v2, t.v2::Float64 as $expr4, t._row_id, Vnode(t._row_id) as $expr5] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test approx_percentile hash_agg forced should use single phase agg + sql: | + SET RW_FORCE_TWO_PHASE_AGG=true; + create table t (v1 int, grp_col int); + select approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t group by grp_col; + stream_error: |- + Feature is not yet implemented: two-phase streaming approx percentile aggregation with group key, please use single phase aggregation instead + No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml - name: test approx percentile with default relative_error sql: | CREATE TABLE t (v1 int); diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index e63b7d760a68..cf7025be9087 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -311,7 +311,10 @@ impl LogicalAgg { col_mapping: approx_percentile_col_mapping, } = approx; if !self.group_key().is_empty() && !approx_percentile_agg_calls.is_empty() { - bail_not_implemented!("two-phase approx percentile agg with group key, please use single phase agg for approx_percentile with group key"); + bail_not_implemented!( + "two-phase streaming approx percentile aggregation with group key, \ + please use single phase aggregation instead" + ); } // Either we have approx percentile aggs and non_approx percentile aggs, From f54c7fe03198f5dd0c8a822763c380414290ce7f Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Mon, 19 Aug 2024 14:27:48 +0800 Subject: [PATCH 14/26] feat: Optimize the performance of scaling in the SQL backend. (#16722) Signed-off-by: Shanicky Chen --- src/meta/model_v2/src/actor_dispatcher.rs | 6 +- src/meta/model_v2/src/lib.rs | 2 +- src/meta/node/src/lib.rs | 1 - src/meta/src/controller/mod.rs | 1 + src/meta/src/controller/scale.rs | 345 ++++++++++++++++++++++ src/meta/src/stream/scale.rs | 266 ++++++++++++++--- 6 files changed, 574 insertions(+), 47 deletions(-) create mode 100644 src/meta/src/controller/scale.rs diff --git a/src/meta/model_v2/src/actor_dispatcher.rs b/src/meta/model_v2/src/actor_dispatcher.rs index 81211cc57270..7d40af6967d3 100644 --- a/src/meta/model_v2/src/actor_dispatcher.rs +++ b/src/meta/model_v2/src/actor_dispatcher.rs @@ -12,13 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::hash::Hash; + use risingwave_pb::stream_plan::{PbDispatcher, PbDispatcherType}; use sea_orm::entity::prelude::*; use serde::{Deserialize, Serialize}; use crate::{ActorId, ActorMapping, FragmentId, I32Array}; -#[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum, Deserialize, Serialize)] +#[derive( + Hash, Copy, Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum, Serialize, Deserialize, +)] #[sea_orm(rs_type = "String", db_type = "String(None)")] pub enum DispatcherType { #[sea_orm(string_value = "HASH")] diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index 23f97e777851..5a386f1ecac3 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -91,7 +91,7 @@ pub type HummockSstableObjectId = i64; pub type FragmentId = i32; pub type ActorId = i32; -#[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum, Serialize, Deserialize)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum, Serialize, Deserialize)] #[sea_orm(rs_type = "String", db_type = "String(None)")] pub enum JobStatus { #[sea_orm(string_value = "INITIAL")] diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index 825d40b5172d..049519372c81 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -296,7 +296,6 @@ pub fn start( ), }, }; - validate_config(&config); let total_memory_bytes = resource_util::memory::system_memory_available_bytes(); diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 43078ea81272..6b56113aa38a 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -38,6 +38,7 @@ pub mod cluster; pub mod fragment; pub mod id; pub mod rename; +pub mod scale; pub mod session_params; pub mod streaming_job; pub mod system_param; diff --git a/src/meta/src/controller/scale.rs b/src/meta/src/controller/scale.rs new file mode 100644 index 000000000000..658ebaef47f6 --- /dev/null +++ b/src/meta/src/controller/scale.rs @@ -0,0 +1,345 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::{HashMap, HashSet}; + +use risingwave_meta_model_migration::{ + Alias, CommonTableExpression, Expr, IntoColumnRef, QueryStatementBuilder, SelectStatement, + UnionType, WithClause, WithQuery, +}; +use risingwave_meta_model_v2::actor_dispatcher::DispatcherType; +use risingwave_meta_model_v2::prelude::{Actor, ActorDispatcher, Fragment, StreamingJob}; +use risingwave_meta_model_v2::{ + actor, actor_dispatcher, fragment, streaming_job, ActorId, FragmentId, ObjectId, +}; +use sea_orm::{ + ColumnTrait, ConnectionTrait, DbErr, EntityTrait, JoinType, QueryFilter, QuerySelect, + RelationTrait, Statement, TransactionTrait, +}; + +use crate::controller::catalog::CatalogController; +use crate::{MetaError, MetaResult}; + +/// This function will construct a query using recursive cte to find `no_shuffle` upstream relation graph for target fragments. +/// +/// # Examples +/// +/// ``` +/// use risingwave_meta::controller::scale::construct_no_shuffle_upstream_traverse_query; +/// use sea_orm::sea_query::*; +/// use sea_orm::*; +/// +/// let query = construct_no_shuffle_upstream_traverse_query(vec![2, 3]); +/// +/// assert_eq!(query.to_string(MysqlQueryBuilder), r#"WITH RECURSIVE `shuffle_deps` (`fragment_id`, `dispatcher_type`, `dispatcher_id`) AS (SELECT DISTINCT `actor`.`fragment_id`, `actor_dispatcher`.`dispatcher_type`, `actor_dispatcher`.`dispatcher_id` FROM `actor` INNER JOIN `actor_dispatcher` ON `actor`.`actor_id` = `actor_dispatcher`.`actor_id` WHERE `actor_dispatcher`.`dispatcher_type` = 'NO_SHUFFLE' AND `actor_dispatcher`.`dispatcher_id` IN (2, 3) UNION ALL (SELECT DISTINCT `actor`.`fragment_id`, `actor_dispatcher`.`dispatcher_type`, `actor_dispatcher`.`dispatcher_id` FROM `actor` INNER JOIN `actor_dispatcher` ON `actor`.`actor_id` = `actor_dispatcher`.`actor_id` INNER JOIN `shuffle_deps` ON `shuffle_deps`.`fragment_id` = `actor_dispatcher`.`dispatcher_id` WHERE `actor_dispatcher`.`dispatcher_type` = 'NO_SHUFFLE')) SELECT DISTINCT `fragment_id`, `dispatcher_type`, `dispatcher_id` FROM `shuffle_deps`"#); +/// assert_eq!(query.to_string(PostgresQueryBuilder), r#"WITH RECURSIVE "shuffle_deps" ("fragment_id", "dispatcher_type", "dispatcher_id") AS (SELECT DISTINCT "actor"."fragment_id", "actor_dispatcher"."dispatcher_type", "actor_dispatcher"."dispatcher_id" FROM "actor" INNER JOIN "actor_dispatcher" ON "actor"."actor_id" = "actor_dispatcher"."actor_id" WHERE "actor_dispatcher"."dispatcher_type" = 'NO_SHUFFLE' AND "actor_dispatcher"."dispatcher_id" IN (2, 3) UNION ALL (SELECT DISTINCT "actor"."fragment_id", "actor_dispatcher"."dispatcher_type", "actor_dispatcher"."dispatcher_id" FROM "actor" INNER JOIN "actor_dispatcher" ON "actor"."actor_id" = "actor_dispatcher"."actor_id" INNER JOIN "shuffle_deps" ON "shuffle_deps"."fragment_id" = "actor_dispatcher"."dispatcher_id" WHERE "actor_dispatcher"."dispatcher_type" = 'NO_SHUFFLE')) SELECT DISTINCT "fragment_id", "dispatcher_type", "dispatcher_id" FROM "shuffle_deps""#); +/// assert_eq!(query.to_string(SqliteQueryBuilder), r#"WITH RECURSIVE "shuffle_deps" ("fragment_id", "dispatcher_type", "dispatcher_id") AS (SELECT DISTINCT "actor"."fragment_id", "actor_dispatcher"."dispatcher_type", "actor_dispatcher"."dispatcher_id" FROM "actor" INNER JOIN "actor_dispatcher" ON "actor"."actor_id" = "actor_dispatcher"."actor_id" WHERE "actor_dispatcher"."dispatcher_type" = 'NO_SHUFFLE' AND "actor_dispatcher"."dispatcher_id" IN (2, 3) UNION ALL SELECT DISTINCT "actor"."fragment_id", "actor_dispatcher"."dispatcher_type", "actor_dispatcher"."dispatcher_id" FROM "actor" INNER JOIN "actor_dispatcher" ON "actor"."actor_id" = "actor_dispatcher"."actor_id" INNER JOIN "shuffle_deps" ON "shuffle_deps"."fragment_id" = "actor_dispatcher"."dispatcher_id" WHERE "actor_dispatcher"."dispatcher_type" = 'NO_SHUFFLE') SELECT DISTINCT "fragment_id", "dispatcher_type", "dispatcher_id" FROM "shuffle_deps""#); +/// ``` +pub fn construct_no_shuffle_upstream_traverse_query(fragment_ids: Vec) -> WithQuery { + construct_no_shuffle_traverse_query_helper(fragment_ids, NoShuffleResolveDirection::Upstream) +} + +pub fn construct_no_shuffle_downstream_traverse_query(fragment_ids: Vec) -> WithQuery { + construct_no_shuffle_traverse_query_helper(fragment_ids, NoShuffleResolveDirection::Downstream) +} + +enum NoShuffleResolveDirection { + Upstream, + Downstream, +} + +fn construct_no_shuffle_traverse_query_helper( + fragment_ids: Vec, + direction: NoShuffleResolveDirection, +) -> WithQuery { + let cte_alias = Alias::new("shuffle_deps"); + + // If we need to look upwards + // resolve by fragment_id -> dispatcher_id + // and if downwards + // resolve by dispatcher_id -> fragment_id + let (cte_ref_column, compared_column) = match direction { + NoShuffleResolveDirection::Upstream => ( + (cte_alias.clone(), actor::Column::FragmentId).into_column_ref(), + (ActorDispatcher, actor_dispatcher::Column::DispatcherId).into_column_ref(), + ), + NoShuffleResolveDirection::Downstream => ( + (cte_alias.clone(), actor_dispatcher::Column::DispatcherId).into_column_ref(), + (Actor, actor::Column::FragmentId).into_column_ref(), + ), + }; + + let mut base_query = SelectStatement::new() + .column((Actor, actor::Column::FragmentId)) + .column((ActorDispatcher, actor_dispatcher::Column::DispatcherType)) + .column((ActorDispatcher, actor_dispatcher::Column::DispatcherId)) + .distinct() + .from(Actor) + .inner_join( + ActorDispatcher, + Expr::col((Actor, actor::Column::ActorId)).eq(Expr::col(( + ActorDispatcher, + actor_dispatcher::Column::ActorId, + ))), + ) + .and_where( + Expr::col((ActorDispatcher, actor_dispatcher::Column::DispatcherType)) + .eq(DispatcherType::NoShuffle), + ) + .and_where(Expr::col(compared_column.clone()).is_in(fragment_ids.clone())) + .to_owned(); + + let cte_referencing = SelectStatement::new() + .column((Actor, actor::Column::FragmentId)) + .column((ActorDispatcher, actor_dispatcher::Column::DispatcherType)) + .column((ActorDispatcher, actor_dispatcher::Column::DispatcherId)) + .distinct() + .from(Actor) + .inner_join( + ActorDispatcher, + Expr::col((Actor, actor::Column::ActorId)).eq(Expr::col(( + ActorDispatcher, + actor_dispatcher::Column::ActorId, + ))), + ) + .inner_join( + cte_alias.clone(), + Expr::col(cte_ref_column).eq(Expr::col(compared_column)), + ) + .and_where( + Expr::col((ActorDispatcher, actor_dispatcher::Column::DispatcherType)) + .eq(DispatcherType::NoShuffle), + ) + .to_owned(); + + let common_table_expr = CommonTableExpression::new() + .query(base_query.union(UnionType::All, cte_referencing).to_owned()) + .column(actor::Column::FragmentId) + .column(actor_dispatcher::Column::DispatcherType) + .column(actor_dispatcher::Column::DispatcherId) + .table_name(cte_alias.clone()) + .to_owned(); + + SelectStatement::new() + .column(actor::Column::FragmentId) + .column(actor_dispatcher::Column::DispatcherType) + .column(actor_dispatcher::Column::DispatcherId) + .distinct() + .from(cte_alias.clone()) + .to_owned() + .with( + WithClause::new() + .recursive(true) + .cte(common_table_expr) + .to_owned(), + ) + .to_owned() +} + +#[derive(Debug, Clone)] +pub struct RescheduleWorkingSet { + pub fragments: HashMap, + pub actors: HashMap, + pub actor_dispatchers: HashMap>, + + pub fragment_downstreams: HashMap>, + pub fragment_upstreams: HashMap>, + + pub related_jobs: HashMap, +} + +async fn resolve_no_shuffle_query( + txn: &C, + query: WithQuery, +) -> MetaResult> +where + C: ConnectionTrait, +{ + let (sql, values) = query.build_any(&*txn.get_database_backend().get_query_builder()); + + let result = txn + .query_all(Statement::from_sql_and_values( + txn.get_database_backend(), + sql, + values, + )) + .await? + .into_iter() + .map(|res| res.try_get_many_by_index()) + .collect::, DbErr>>() + .map_err(MetaError::from)?; + + Ok(result) +} + +impl CatalogController { + pub async fn resolve_working_set_for_reschedule_fragments( + &self, + fragment_ids: Vec, + ) -> MetaResult { + let inner = self.inner.read().await; + self.resolve_working_set_for_reschedule_helper(&inner.db, fragment_ids) + .await + } + + pub async fn resolve_working_set_for_reschedule_tables( + &self, + table_ids: Vec, + ) -> MetaResult { + let inner = self.inner.read().await; + let txn = inner.db.begin().await?; + + let fragment_ids: Vec = Fragment::find() + .filter(fragment::Column::JobId.is_in(table_ids)) + .all(&txn) + .await? + .into_iter() + .map(|fragment| fragment.fragment_id) + .collect(); + + self.resolve_working_set_for_reschedule_helper(&txn, fragment_ids) + .await + } + + pub async fn resolve_working_set_for_reschedule_helper( + &self, + txn: &C, + fragment_ids: Vec, + ) -> MetaResult + where + C: ConnectionTrait, + { + // NO_SHUFFLE related multi-layer upstream fragments + let no_shuffle_related_upstream_fragment_ids = resolve_no_shuffle_query( + txn, + construct_no_shuffle_upstream_traverse_query(fragment_ids.clone()), + ) + .await?; + + // NO_SHUFFLE related multi-layer downstream fragments + let no_shuffle_related_downstream_fragment_ids = resolve_no_shuffle_query( + txn, + construct_no_shuffle_downstream_traverse_query(fragment_ids.clone()), + ) + .await?; + + // We need to identify all other types of dispatchers that are Leaves in the NO_SHUFFLE dependency tree. + let extended_fragment_ids: HashSet<_> = no_shuffle_related_upstream_fragment_ids + .iter() + .chain(no_shuffle_related_downstream_fragment_ids.iter()) + .flat_map(|(src, _, dst)| [*src, *dst]) + .chain(fragment_ids.iter().cloned()) + .collect(); + + let query = Actor::find() + .select_only() + .column(actor::Column::FragmentId) + .column(actor_dispatcher::Column::DispatcherType) + .column(actor_dispatcher::Column::DispatcherId) + .distinct() + .join(JoinType::InnerJoin, actor::Relation::ActorDispatcher.def()); + + // single-layer upstream fragment ids + let upstream_fragments: Vec<(FragmentId, DispatcherType, FragmentId)> = query + .clone() + .filter(actor_dispatcher::Column::DispatcherId.is_in(extended_fragment_ids.clone())) + .into_tuple() + .all(txn) + .await?; + + // single-layer downstream fragment ids + let downstream_fragments: Vec<(FragmentId, DispatcherType, FragmentId)> = query + .clone() + .filter(actor::Column::FragmentId.is_in(extended_fragment_ids.clone())) + .into_tuple() + .all(txn) + .await?; + + let all_fragment_relations: HashSet<_> = no_shuffle_related_upstream_fragment_ids + .into_iter() + .chain(no_shuffle_related_downstream_fragment_ids.into_iter()) + .chain(upstream_fragments.into_iter()) + .chain(downstream_fragments.into_iter()) + .collect(); + + let mut fragment_upstreams: HashMap> = + HashMap::new(); + let mut fragment_downstreams: HashMap> = + HashMap::new(); + + for (src, dispatcher_type, dst) in &all_fragment_relations { + fragment_upstreams + .entry(*dst) + .or_default() + .push((*src, *dispatcher_type)); + fragment_downstreams + .entry(*src) + .or_default() + .push((*dst, *dispatcher_type)); + } + + let all_fragment_ids: HashSet<_> = all_fragment_relations + .iter() + .flat_map(|(src, _, dst)| [*src, *dst]) + .chain(extended_fragment_ids.into_iter()) + .collect(); + + let fragments: Vec<_> = Fragment::find() + .filter(fragment::Column::FragmentId.is_in(all_fragment_ids.clone())) + .all(txn) + .await?; + + let actor_and_dispatchers: Vec<(_, _)> = Actor::find() + .filter(actor::Column::FragmentId.is_in(all_fragment_ids.clone())) + .find_with_related(ActorDispatcher) + .all(txn) + .await?; + + let mut actors = HashMap::with_capacity(actor_and_dispatchers.len()); + let mut actor_dispatchers = HashMap::with_capacity(actor_and_dispatchers.len()); + + for (actor, dispatchers) in actor_and_dispatchers { + let actor_id = actor.actor_id; + actors.insert(actor_id, actor); + actor_dispatchers.insert(actor_id, dispatchers); + } + + let fragments: HashMap = fragments + .into_iter() + .map(|fragment| (fragment.fragment_id, fragment)) + .collect(); + + let related_job_ids: HashSet<_> = + fragments.values().map(|fragment| fragment.job_id).collect(); + + let related_jobs = StreamingJob::find() + .filter(streaming_job::Column::JobId.is_in(related_job_ids)) + .all(txn) + .await?; + + let related_jobs = related_jobs + .into_iter() + .map(|job| (job.job_id, job)) + .collect(); + + Ok(RescheduleWorkingSet { + fragments, + actors, + actor_dispatchers, + fragment_downstreams, + fragment_upstreams, + related_jobs, + }) + } +} diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index f61789c8f937..885fa27d043e 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -30,7 +30,7 @@ use risingwave_common::bitmap::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::TableId; use risingwave_common::hash::{ActorMapping, VirtualNode}; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_meta_model_v2::StreamingParallelism; +use risingwave_meta_model_v2::{actor, fragment, ObjectId, StreamingParallelism}; use risingwave_pb::common::{ActorInfo, Buffer, PbActorLocation, WorkerNode, WorkerType}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::table_fragments::actor_status::ActorState; @@ -41,7 +41,7 @@ use risingwave_pb::meta::table_fragments::{self, ActorStatus, PbFragment, State} use risingwave_pb::meta::FragmentWorkerSlotMappings; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ - Dispatcher, DispatcherType, FragmentTypeFlag, PbStreamActor, StreamNode, + Dispatcher, DispatcherType, FragmentTypeFlag, PbDispatcher, PbStreamActor, StreamNode, }; use risingwave_pb::stream_service::build_actor_info::SubscriptionIds; use risingwave_pb::stream_service::BuildActorInfo; @@ -52,8 +52,10 @@ use tokio::task::JoinHandle; use tokio::time::{Instant, MissedTickBehavior}; use crate::barrier::{Command, Reschedule, StreamRpcManager}; +use crate::controller::scale::RescheduleWorkingSet; use crate::manager::{ - IdCategory, IdGenManagerImpl, LocalNotification, MetaSrvEnv, MetadataManager, WorkerId, + IdCategory, IdGenManagerImpl, LocalNotification, MetaSrvEnv, MetadataManager, + MetadataManagerV2, WorkerId, }; use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments, TableParallelism}; use crate::serving::{ @@ -61,7 +63,7 @@ use crate::serving::{ }; use crate::storage::{MetaStore, MetaStoreError, MetaStoreRef, Transaction, DEFAULT_COLUMN_FAMILY}; use crate::stream::{GlobalStreamManager, SourceManagerRef}; -use crate::{model, MetaError, MetaResult}; +use crate::{MetaError, MetaResult}; #[derive(Default, Copy, Clone, Debug, Eq, PartialEq, Ord, PartialOrd)] pub struct TableRevision(u64); @@ -120,7 +122,7 @@ pub struct CustomFragmentInfo { pub actors: Vec, } -#[derive(Default)] +#[derive(Default, Clone)] pub struct CustomActorInfo { pub actor_id: u32, pub fragment_id: u32, @@ -540,6 +542,141 @@ impl ScaleController { ); } + async fn fulfill_index_by_fragment_ids( + actor_map: &mut HashMap, + fragment_map: &mut HashMap, + actor_status: &mut BTreeMap, + fragment_state: &mut HashMap, + fragment_to_table: &mut HashMap, + mgr: &MetadataManagerV2, + fragment_ids: Vec, + ) -> Result<(), MetaError> { + let RescheduleWorkingSet { + fragments, + actors, + mut actor_dispatchers, + fragment_downstreams: _, + fragment_upstreams: _, + related_jobs, + } = mgr + .catalog_controller + .resolve_working_set_for_reschedule_fragments(fragment_ids) + .await?; + + let mut fragment_actors: HashMap< + risingwave_meta_model_v2::FragmentId, + Vec, + > = HashMap::new(); + + let mut expr_contexts = HashMap::new(); + for ( + _, + actor::Model { + actor_id, + fragment_id, + status: _, + splits: _, + worker_id, + upstream_actor_ids, + vnode_bitmap, + expr_context, + }, + ) in actors + { + let dispatchers = actor_dispatchers + .remove(&actor_id) + .unwrap_or_default() + .into_iter() + .map(PbDispatcher::from) + .collect(); + + let actor_info = CustomActorInfo { + actor_id: actor_id as _, + fragment_id: fragment_id as _, + dispatcher: dispatchers, + upstream_actor_id: upstream_actor_ids + .into_inner() + .values() + .flatten() + .map(|id| *id as _) + .collect(), + vnode_bitmap: vnode_bitmap.map(|bitmap| bitmap.to_protobuf()), + }; + + actor_map.insert(actor_id as _, actor_info.clone()); + + fragment_actors + .entry(fragment_id as _) + .or_default() + .push(actor_info); + + actor_status.insert(actor_id as _, worker_id as WorkerId); + + expr_contexts.insert(actor_id as u32, expr_context); + } + + for ( + _, + fragment::Model { + fragment_id, + job_id, + fragment_type_mask, + distribution_type, + stream_node, + state_table_ids, + upstream_fragment_id, + }, + ) in fragments + { + let actors = fragment_actors + .remove(&(fragment_id as _)) + .unwrap_or_default(); + + let CustomActorInfo { + actor_id, + fragment_id, + dispatcher, + upstream_actor_id, + vnode_bitmap, + } = actors.first().unwrap().clone(); + + let fragment = CustomFragmentInfo { + fragment_id: fragment_id as _, + fragment_type_mask: fragment_type_mask as _, + distribution_type: distribution_type.into(), + state_table_ids: state_table_ids.into_u32_array(), + upstream_fragment_ids: upstream_fragment_id.into_u32_array(), + actor_template: PbStreamActor { + nodes: Some(stream_node.to_protobuf()), + actor_id, + fragment_id: fragment_id as _, + dispatcher, + upstream_actor_id, + vnode_bitmap, + // todo, we need to fill this part + mview_definition: "".to_string(), + expr_context: expr_contexts + .get(&actor_id) + .cloned() + .map(|expr_context| expr_context.to_protobuf()), + }, + actors, + }; + + fragment_map.insert(fragment_id as _, fragment); + + fragment_to_table.insert(fragment_id as _, TableId::from(job_id as u32)); + + let related_job = related_jobs.get(&job_id).expect("job not found"); + + fragment_state.insert( + fragment_id, + table_fragments::PbState::from(related_job.job_status), + ); + } + Ok(()) + } + match &self.metadata_manager { MetadataManager::V1(mgr) => { let guard = mgr.fragment_manager.get_fragment_read_guard().await; @@ -555,19 +692,19 @@ impl ScaleController { ); } } - MetadataManager::V2(_) => { - let all_table_fragments = self.list_all_table_fragments().await?; - - for table_fragments in &all_table_fragments { - fulfill_index_by_table_fragments_ref( - &mut actor_map, - &mut fragment_map, - &mut actor_status, - &mut fragment_state, - &mut fragment_to_table, - table_fragments, - ); - } + MetadataManager::V2(mgr) => { + let fragment_ids = reschedule.keys().map(|id| *id as _).collect(); + + fulfill_index_by_fragment_ids( + &mut actor_map, + &mut fragment_map, + &mut actor_status, + &mut fragment_state, + &mut fragment_to_table, + mgr, + fragment_ids, + ) + .await?; } }; @@ -1814,23 +1951,6 @@ impl ScaleController { Ok(()) } - // FIXME: should be removed - pub(crate) async fn list_all_table_fragments(&self) -> MetaResult> { - use crate::model::MetadataModel; - let all_table_fragments = match &self.metadata_manager { - MetadataManager::V1(mgr) => mgr.fragment_manager.list_table_fragments().await, - MetadataManager::V2(mgr) => mgr - .catalog_controller - .table_fragments() - .await? - .into_values() - .map(model::TableFragments::from_protobuf) - .collect(), - }; - - Ok(all_table_fragments) - } - pub async fn generate_table_resize_plan( &self, policy: TableResizePolicy, @@ -1962,6 +2082,62 @@ impl ScaleController { Ok(()) } + async fn build_index_v2( + no_shuffle_source_fragment_ids: &mut HashSet, + no_shuffle_target_fragment_ids: &mut HashSet, + fragment_distribution_map: &mut HashMap, + actor_location: &mut HashMap, + table_fragment_id_map: &mut HashMap>, + fragment_actor_id_map: &mut HashMap>, + mgr: &MetadataManagerV2, + table_ids: Vec, + ) -> Result<(), MetaError> { + let RescheduleWorkingSet { + fragments, + actors, + actor_dispatchers: _actor_dispatchers, + fragment_downstreams, + fragment_upstreams: _fragment_upstreams, + related_jobs: _related_jobs, + } = mgr + .catalog_controller + .resolve_working_set_for_reschedule_tables(table_ids) + .await?; + + for (fragment_id, downstreams) in fragment_downstreams { + for (downstream_fragment_id, dispatcher_type) in downstreams { + if let risingwave_meta_model_v2::actor_dispatcher::DispatcherType::NoShuffle = + dispatcher_type + { + no_shuffle_source_fragment_ids.insert(fragment_id as FragmentId); + no_shuffle_target_fragment_ids.insert(downstream_fragment_id as FragmentId); + } + } + } + + for (fragment_id, fragment) in fragments { + fragment_distribution_map.insert( + fragment_id as FragmentId, + FragmentDistributionType::from(fragment.distribution_type), + ); + + table_fragment_id_map + .entry(fragment.job_id as u32) + .or_default() + .insert(fragment_id as FragmentId); + } + + for (actor_id, actor) in actors { + actor_location.insert(actor_id as ActorId, actor.worker_id as WorkerId); + fragment_actor_id_map + .entry(actor.fragment_id as FragmentId) + .or_default() + .insert(actor_id as ActorId); + } + + Ok(()) + } + match &self.metadata_manager { MetadataManager::V1(mgr) => { let guard = mgr.fragment_manager.get_fragment_read_guard().await; @@ -1975,22 +2151,24 @@ impl ScaleController { guard.table_fragments(), )?; } - MetadataManager::V2(_) => { - let all_table_fragments = self.list_all_table_fragments().await?; - let all_table_fragments = all_table_fragments - .into_iter() - .map(|table_fragments| (table_fragments.table_id(), table_fragments)) - .collect::>(); - build_index( + MetadataManager::V2(mgr) => { + let table_ids = table_parallelisms + .keys() + .map(|id| *id as ObjectId) + .collect(); + + build_index_v2( &mut no_shuffle_source_fragment_ids, &mut no_shuffle_target_fragment_ids, &mut fragment_distribution_map, &mut actor_location, &mut table_fragment_id_map, &mut fragment_actor_id_map, - &all_table_fragments, - )?; + mgr, + table_ids, + ) + .await?; } } From 09865088ffd0077779665e8d409a3e6fe67d1ee4 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 19 Aug 2024 14:40:12 +0800 Subject: [PATCH 15/26] refactor(meta): only retry on connection error when registering worker node (#18061) Signed-off-by: Bugen Zhao --- proto/meta.proto | 3 - src/compute/src/server.rs | 3 +- src/ctl/src/common/meta_service.rs | 2 +- src/frontend/src/session.rs | 2 +- src/meta/service/src/cluster_service.rs | 30 +--- src/rpc_client/src/error.rs | 20 +++ src/rpc_client/src/meta_client.rs | 130 +++++++++++------- src/storage/compactor/src/server.rs | 3 +- .../src/compaction_test_runner.rs | 8 +- 9 files changed, 115 insertions(+), 86 deletions(-) diff --git a/proto/meta.proto b/proto/meta.proto index bcb6c331549f..4f51522cc006 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -321,9 +321,6 @@ message AddWorkerNodeRequest { } message AddWorkerNodeResponse { - reserved 3; - reserved "system_params"; - common.Status status = 1; optional uint32 node_id = 2; string cluster_id = 4; } diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 3270897c29d3..c850f839e620 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -130,8 +130,7 @@ pub async fn compute_node_serve( }, &config.meta, ) - .await - .unwrap(); + .await; let state_store_url = system_params.state_store(); diff --git a/src/ctl/src/common/meta_service.rs b/src/ctl/src/common/meta_service.rs index ac539b9233ea..6d70bdf94283 100644 --- a/src/ctl/src/common/meta_service.rs +++ b/src/ctl/src/common/meta_service.rs @@ -62,7 +62,7 @@ Note: the default value of `RW_META_ADDR` is 'http://127.0.0.1:5690'."; Property::default(), &MetaConfig::default(), ) - .await?; + .await; let worker_id = client.worker_id(); tracing::info!("registered as RiseCtl worker, worker_id = {}", worker_id); Ok(client) diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index b970899ef080..685000dbbe65 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -267,7 +267,7 @@ impl FrontendEnv { Default::default(), &config.meta, ) - .await?; + .await; let worker_id = meta_client.worker_id(); info!("Assigned worker node id {}", worker_id); diff --git a/src/meta/service/src/cluster_service.rs b/src/meta/service/src/cluster_service.rs index 39cd40ed3740..6f1bfd68e5b3 100644 --- a/src/meta/service/src/cluster_service.rs +++ b/src/meta/service/src/cluster_service.rs @@ -25,7 +25,6 @@ use risingwave_pb::meta::{ ListAllNodesResponse, UpdateWorkerNodeSchedulabilityRequest, UpdateWorkerNodeSchedulabilityResponse, }; -use thiserror_ext::AsReport; use tonic::{Request, Response, Status}; use crate::MetaError; @@ -58,31 +57,16 @@ impl ClusterService for ClusterServiceImpl { .property .ok_or_else(|| MetaError::invalid_parameter("worker node property is not provided"))?; let resource = req.resource.unwrap_or_default(); - let result = self + let worker_id = self .metadata_manager .add_worker_node(worker_type, host, property, resource) - .await; + .await?; let cluster_id = self.metadata_manager.cluster_id().to_string(); - match result { - Ok(worker_id) => Ok(Response::new(AddWorkerNodeResponse { - status: None, - node_id: Some(worker_id), - cluster_id, - })), - Err(e) => { - if e.is_invalid_worker() { - return Ok(Response::new(AddWorkerNodeResponse { - status: Some(risingwave_pb::common::Status { - code: risingwave_pb::common::status::Code::UnknownWorker as i32, - message: e.to_report_string(), - }), - node_id: None, - cluster_id, - })); - } - Err(e.into()) - } - } + + Ok(Response::new(AddWorkerNodeResponse { + node_id: Some(worker_id), + cluster_id, + })) } /// Update schedulability of a compute node. Will not affect actors which are already running on diff --git a/src/rpc_client/src/error.rs b/src/rpc_client/src/error.rs index c5c5613a32a4..26204cc1908c 100644 --- a/src/rpc_client/src/error.rs +++ b/src/rpc_client/src/error.rs @@ -77,3 +77,23 @@ macro_rules! impl_from_status { } impl_from_status!(stream, batch, meta, compute, compactor, connector); + +impl RpcError { + /// Returns `true` if the error is a connection error. Typically used to determine if + /// the error is transient and can be retried. + pub fn is_connection_error(&self) -> bool { + match self { + RpcError::TransportError(_) => true, + RpcError::GrpcStatus(status) => matches!( + status.inner().code(), + tonic::Code::Unavailable // server not started + | tonic::Code::Unknown // could be transport error + | tonic::Code::Unimplemented // meta leader service not started + ), + RpcError::MetaAddressParse(_) => false, + RpcError::Internal(anyhow) => anyhow + .downcast_ref::() // this skips all contexts attached to the error + .map_or(false, Self::is_connection_error), + } + } +} diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 5a45e0752c9d..ae99c57afd7c 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -36,6 +36,7 @@ use risingwave_common::util::meta_addr::MetaAddressStrategy; use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::RW_VERSION; +use risingwave_error::bail; use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{ @@ -220,12 +221,33 @@ impl MetaClient { } /// Register the current node to the cluster and set the corresponding worker id. + /// + /// Retry if there's connection issue with the meta node. Exit the process if the registration fails. pub async fn register_new( addr_strategy: MetaAddressStrategy, worker_type: WorkerType, addr: &HostAddr, property: Property, meta_config: &MetaConfig, + ) -> (Self, SystemParamsReader) { + let ret = + Self::register_new_inner(addr_strategy, worker_type, addr, property, meta_config).await; + + match ret { + Ok(ret) => ret, + Err(err) => { + tracing::error!(error = %err.as_report(), "failed to register worker, exiting..."); + std::process::exit(1); + } + } + } + + async fn register_new_inner( + addr_strategy: MetaAddressStrategy, + worker_type: WorkerType, + addr: &HostAddr, + property: Property, + meta_config: &MetaConfig, ) -> Result<(Self, SystemParamsReader)> { tracing::info!("register meta client using strategy: {}", addr_strategy); @@ -238,34 +260,35 @@ impl MetaClient { if property.is_unschedulable { tracing::warn!("worker {:?} registered as unschedulable", addr.clone()); } - let init_result: Result<_> = tokio_retry::Retry::spawn(retry_strategy, || async { - let grpc_meta_client = GrpcMetaClient::new(&addr_strategy, meta_config.clone()).await?; - - let add_worker_resp = grpc_meta_client - .add_worker_node(AddWorkerNodeRequest { - worker_type: worker_type as i32, - host: Some(addr.to_protobuf()), - property: Some(property), - resource: Some(risingwave_pb::common::worker_node::Resource { - rw_version: RW_VERSION.to_string(), - total_memory_bytes: system_memory_available_bytes() as _, - total_cpu_cores: total_cpu_available() as _, - }), - }) - .await?; - if let Some(status) = &add_worker_resp.status - && status.code() == risingwave_pb::common::status::Code::UnknownWorker - { - tracing::error!("invalid worker: {}", status.message); - std::process::exit(1); - } + let init_result: Result<_> = tokio_retry::RetryIf::spawn( + retry_strategy, + || async { + let grpc_meta_client = + GrpcMetaClient::new(&addr_strategy, meta_config.clone()).await?; + + let add_worker_resp = grpc_meta_client + .add_worker_node(AddWorkerNodeRequest { + worker_type: worker_type as i32, + host: Some(addr.to_protobuf()), + property: Some(property), + resource: Some(risingwave_pb::common::worker_node::Resource { + rw_version: RW_VERSION.to_string(), + total_memory_bytes: system_memory_available_bytes() as _, + total_cpu_cores: total_cpu_available() as _, + }), + }) + .await + .context("failed to add worker node")?; - let system_params_resp = grpc_meta_client - .get_system_params(GetSystemParamsRequest {}) - .await?; + let system_params_resp = grpc_meta_client + .get_system_params(GetSystemParamsRequest {}) + .await + .context("failed to get initial system params")?; - Ok((add_worker_resp, system_params_resp, grpc_meta_client)) - }) + Ok((add_worker_resp, system_params_resp, grpc_meta_client)) + }, + RpcError::is_connection_error, + ) .await; let (add_worker_resp, system_params_resp, grpc_meta_client) = init_result?; @@ -1708,38 +1731,40 @@ impl MetaMemberManagement { let mut fetched_members = None; for (addr, client) in &mut member_group.members { - let client: Result = try { - match client { + let members: Result<_> = try { + let mut client = match client { Some(cached_client) => cached_client.to_owned(), None => { let endpoint = GrpcMetaClient::addr_to_endpoint(addr.clone()); - let channel = GrpcMetaClient::connect_to_endpoint(endpoint).await?; + let channel = GrpcMetaClient::connect_to_endpoint(endpoint) + .await + .context("failed to create client")?; let new_client: MetaMemberClient = MetaMemberServiceClient::new(channel); *client = Some(new_client.clone()); new_client } - } + }; + + let resp = client + .members(MembersRequest {}) + .await + .context("failed to fetch members")?; + + resp.into_inner().members }; - if let Err(err) = client { - tracing::warn!(%addr, error = %err.as_report(), "failed to create client"); - continue; - } - match client.unwrap().members(MembersRequest {}).await { - Err(err) => { - tracing::warn!(%addr, error = %err.as_report(), "failed to fetch members"); - continue; - } - Ok(resp) => { - fetched_members = Some(resp.into_inner().members); - break; - } + + let fetched = members.is_ok(); + fetched_members = Some(members); + if fetched { + break; } } - let members = - fetched_members.ok_or_else(|| anyhow!("could not refresh members"))?; + let members = fetched_members + .context("no member available in the list")? + .context("could not refresh members")?; // find new leader let mut leader = None; @@ -1916,7 +1941,7 @@ impl GrpcMetaClient { .map(|addr| (Self::addr_to_endpoint(addr.clone()), addr)) .collect(); - let endpoints = endpoints.clone(); + let mut last_error = None; for (endpoint, addr) in endpoints { match Self::connect_to_endpoint(endpoint).await { @@ -1929,14 +1954,19 @@ impl GrpcMetaClient { error = %e.as_report(), "Failed to connect to meta server {}, trying again", addr, - ) + ); + last_error = Some(e); } } } - Err(RpcError::Internal(anyhow!( - "Failed to connect to meta server" - ))) + if let Some(last_error) = last_error { + Err(anyhow::anyhow!(last_error) + .context("failed to connect to all meta servers") + .into()) + } else { + bail!("no meta server address provided") + } } async fn connect_to_endpoint(endpoint: Endpoint) -> Result { diff --git a/src/storage/compactor/src/server.rs b/src/storage/compactor/src/server.rs index e139bc201cd4..0b086ee06fe1 100644 --- a/src/storage/compactor/src/server.rs +++ b/src/storage/compactor/src/server.rs @@ -197,8 +197,7 @@ pub async fn compactor_serve( Default::default(), &config.meta, ) - .await - .unwrap(); + .await; info!("Assigned compactor id {}", meta_client.worker_id()); diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index 0aa7d1d83c8d..328c23f8fbe8 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -238,7 +238,7 @@ async fn init_metadata_for_replay( std::process::exit(0); }, ret = MetaClient::register_new(cluster_meta_endpoint.parse()?, WorkerType::RiseCtl, advertise_addr, Default::default(), &meta_config) => { - (meta_client, _) = ret.unwrap(); + (meta_client, _) = ret; }, } let worker_id = meta_client.worker_id(); @@ -254,7 +254,7 @@ async fn init_metadata_for_replay( Default::default(), &meta_config, ) - .await?; + .await; new_meta_client.activate(advertise_addr).await.unwrap(); if ci_mode { let table_to_check = tables.iter().find(|t| t.name == "nexmark_q7").unwrap(); @@ -286,7 +286,7 @@ async fn pull_version_deltas( Default::default(), &MetaConfig::default(), ) - .await?; + .await; let worker_id = meta_client.worker_id(); tracing::info!("Assigned pull worker id {}", worker_id); meta_client.activate(advertise_addr).await.unwrap(); @@ -335,7 +335,7 @@ async fn start_replay( Default::default(), &config.meta, ) - .await?; + .await; let worker_id = meta_client.worker_id(); tracing::info!("Assigned replay worker id {}", worker_id); meta_client.activate(&advertise_addr).await.unwrap(); From 7b97788491a5fb8389c7ad0f849810ad591efc57 Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Mon, 19 Aug 2024 15:01:35 +0800 Subject: [PATCH 16/26] fix(schedule): Restrict parallelism that exceed the virtual node limit during automatic scaling. (#18006) Signed-off-by: Shanicky Chen --- src/frontend/src/handler/alter_parallelism.rs | 37 +++++++- src/meta/src/rpc/ddl_controller.rs | 22 +++-- src/meta/src/stream/scale.rs | 48 ++++++++-- src/prost/src/lib.rs | 9 ++ .../scale/auto_parallelism.rs | 2 +- .../scale/streaming_parallelism.rs | 95 +++++++++++++++++++ 6 files changed, 190 insertions(+), 23 deletions(-) diff --git a/src/frontend/src/handler/alter_parallelism.rs b/src/frontend/src/handler/alter_parallelism.rs index e5d2d3303789..3c6ab52f51e3 100644 --- a/src/frontend/src/handler/alter_parallelism.rs +++ b/src/frontend/src/handler/alter_parallelism.rs @@ -14,8 +14,9 @@ use pgwire::pg_response::StatementType; use risingwave_common::bail; +use risingwave_common::hash::VirtualNode; use risingwave_pb::meta::table_parallelism::{ - AdaptiveParallelism, FixedParallelism, PbParallelism, + AdaptiveParallelism, FixedParallelism, Parallelism, PbParallelism, }; use risingwave_pb::meta::{PbTableParallelism, TableParallelism}; use risingwave_sqlparser::ast::{ObjectName, SetVariableValue, SetVariableValueSingle, Value}; @@ -92,15 +93,43 @@ pub async fn handle_alter_parallelism( } }; - let target_parallelism = extract_table_parallelism(parallelism)?; + let mut target_parallelism = extract_table_parallelism(parallelism)?; + + let available_parallelism = session + .env() + .worker_node_manager() + .list_worker_nodes() + .iter() + .filter(|w| w.is_streaming_schedulable()) + .map(|w| w.parallelism) + .sum::(); + + let mut builder = RwPgResponse::builder(stmt_type); + + match &target_parallelism.parallelism { + Some(Parallelism::Adaptive(_)) | Some(Parallelism::Auto(_)) => { + if available_parallelism > VirtualNode::COUNT as u32 { + builder = builder.notice(format!("Available parallelism exceeds the maximum parallelism limit, the actual parallelism will be limited to {}", VirtualNode::COUNT)); + } + } + Some(Parallelism::Fixed(FixedParallelism { parallelism })) => { + if *parallelism > VirtualNode::COUNT as u32 { + builder = builder.notice(format!("Provided parallelism exceeds the maximum parallelism limit, resetting to FIXED({})", VirtualNode::COUNT)); + target_parallelism = PbTableParallelism { + parallelism: Some(PbParallelism::Fixed(FixedParallelism { + parallelism: VirtualNode::COUNT as u32, + })), + }; + } + } + _ => {} + }; let catalog_writer = session.catalog_writer()?; catalog_writer .alter_parallelism(table_id, target_parallelism, deferred) .await?; - let mut builder = RwPgResponse::builder(stmt_type); - if deferred { builder = builder.notice("DEFERRED is used, please ensure that automatic parallelism control is enabled on the meta, otherwise, the alter will not take effect.".to_string()); } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 72d135faa6a0..8062675156fe 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -1523,8 +1523,6 @@ impl DdlController { specified_parallelism: Option, cluster_info: &StreamingClusterInfo, ) -> MetaResult { - const MAX_PARALLELISM: NonZeroUsize = NonZeroUsize::new(VirtualNode::COUNT).unwrap(); - let available_parallelism = cluster_info.parallelism(); if available_parallelism == 0 { return Err(MetaError::unavailable("No available slots to schedule")); @@ -1546,12 +1544,7 @@ impl DdlController { ))); } - if available_parallelism > MAX_PARALLELISM { - tracing::warn!("Too many parallelism, use {} instead", MAX_PARALLELISM); - Ok(MAX_PARALLELISM) - } else { - Ok(parallelism) - } + Ok(parallelism) } /// Builds the actor graph: @@ -1617,6 +1610,15 @@ impl DdlController { let parallelism = self.resolve_stream_parallelism(specified_parallelism, &cluster_info)?; + const MAX_PARALLELISM: NonZeroUsize = NonZeroUsize::new(VirtualNode::COUNT).unwrap(); + + let parallelism_limited = parallelism > MAX_PARALLELISM; + if parallelism_limited { + tracing::warn!("Too many parallelism, use {} instead", MAX_PARALLELISM); + } + + let parallelism = parallelism.min(MAX_PARALLELISM); + let actor_graph_builder = ActorGraphBuilder::new(id, complete_graph, cluster_info, parallelism)?; @@ -1638,6 +1640,10 @@ impl DdlController { // If the frontend does not specify the degree of parallelism and the default_parallelism is set to full, then set it to ADAPTIVE. // Otherwise, it defaults to FIXED based on deduction. let table_parallelism = match (specified_parallelism, &self.env.opts.default_parallelism) { + (None, DefaultParallelism::Full) if parallelism_limited => { + tracing::warn!("Parallelism limited to 256 in ADAPTIVE mode"); + TableParallelism::Adaptive + } (None, DefaultParallelism::Full) => TableParallelism::Adaptive, _ => TableParallelism::Fixed(parallelism.get()), }; diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 885fa27d043e..42ed98b372c7 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -1983,7 +1983,7 @@ impl ScaleController { .map(|worker| (worker.id, worker)) .collect(); - let worker_slots = workers + let schedulable_worker_slots = workers .values() .map(|worker| (worker.id, worker.parallelism as usize)) .collect::>(); @@ -2191,7 +2191,7 @@ impl ScaleController { *fragment_slots.entry(*worker_id).or_default() += 1; } - let all_available_slots: usize = worker_slots.values().cloned().sum(); + let all_available_slots: usize = schedulable_worker_slots.values().cloned().sum(); if all_available_slots == 0 { bail!( @@ -2208,12 +2208,13 @@ impl ScaleController { assert_eq!(*should_be_one, 1); - if worker_slots.contains_key(single_worker_id) { + if schedulable_worker_slots.contains_key(single_worker_id) { // NOTE: shall we continue? continue; } - let units = schedule_units_for_slots(&worker_slots, 1, table_id)?; + let units = + schedule_units_for_slots(&schedulable_worker_slots, 1, table_id)?; let (chosen_target_worker_id, should_be_one) = units.iter().exactly_one().ok().with_context(|| { @@ -2237,14 +2238,41 @@ impl ScaleController { } FragmentDistributionType::Hash => match parallelism { TableParallelism::Adaptive => { - target_plan.insert( - fragment_id, - Self::diff_worker_slot_changes(&fragment_slots, &worker_slots), - ); + if all_available_slots > VirtualNode::COUNT { + tracing::warn!("available parallelism for table {table_id} is larger than VirtualNode::COUNT, force limit to VirtualNode::COUNT"); + // force limit to VirtualNode::COUNT + let target_worker_slots = schedule_units_for_slots( + &schedulable_worker_slots, + VirtualNode::COUNT, + table_id, + )?; + + target_plan.insert( + fragment_id, + Self::diff_worker_slot_changes( + &fragment_slots, + &target_worker_slots, + ), + ); + } else { + target_plan.insert( + fragment_id, + Self::diff_worker_slot_changes( + &fragment_slots, + &schedulable_worker_slots, + ), + ); + } } - TableParallelism::Fixed(n) => { + TableParallelism::Fixed(mut n) => { + if n > VirtualNode::COUNT { + // This should be unreachable, but we still intercept it to prevent accidental modifications. + tracing::warn!("parallelism {n} for table {table_id} is larger than VirtualNode::COUNT, force limit to VirtualNode::COUNT"); + n = VirtualNode::COUNT + } + let target_worker_slots = - schedule_units_for_slots(&worker_slots, n, table_id)?; + schedule_units_for_slots(&schedulable_worker_slots, n, table_id)?; target_plan.insert( fragment_id, diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index b8cab9006dea..0b416af39b83 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -23,6 +23,7 @@ pub use prost::Message; use risingwave_error::tonic::ToTonicStatus; use thiserror::Error; + #[rustfmt::skip] #[cfg_attr(madsim, path = "sim/catalog.rs")] pub mod catalog; @@ -242,6 +243,14 @@ impl meta::table_fragments::ActorStatus { } } +impl common::WorkerNode { + pub fn is_streaming_schedulable(&self) -> bool { + let property = self.property.as_ref(); + property.map_or(false, |p| p.is_streaming) + && !property.map_or(false, |p| p.is_unschedulable) + } +} + impl common::ActorLocation { pub fn from_worker(worker_node_id: u32) -> Option { Some(Self { worker_node_id }) diff --git a/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs b/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs index 8e2b0f446550..b54a5bd3d6b3 100644 --- a/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs +++ b/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs @@ -25,7 +25,7 @@ use risingwave_simulation::utils::AssertResult; use tokio::time::sleep; /// Please ensure that this value is the same as the one in the `risingwave-auto-scale.toml` file. -const MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE: u64 = 15; +pub const MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE: u64 = 15; #[tokio::test] async fn test_passive_online_and_offline() -> Result<()> { diff --git a/src/tests/simulation/tests/integration_tests/scale/streaming_parallelism.rs b/src/tests/simulation/tests/integration_tests/scale/streaming_parallelism.rs index 625a9bc6a8bf..bef6b1ca1f1c 100644 --- a/src/tests/simulation/tests/integration_tests/scale/streaming_parallelism.rs +++ b/src/tests/simulation/tests/integration_tests/scale/streaming_parallelism.rs @@ -12,9 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::time::Duration; + use anyhow::Result; +use madsim::time::sleep; +use risingwave_common::hash::VirtualNode; use risingwave_simulation::cluster::{Cluster, Configuration}; use risingwave_simulation::ctl_ext::predicate::identity_contains; +use risingwave_simulation::utils::AssertResult; + +use crate::scale::auto_parallelism::MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE; #[tokio::test] async fn test_streaming_parallelism_default() -> Result<()> { @@ -135,3 +142,91 @@ async fn test_streaming_parallelism_index() -> Result<()> { ); Ok(()) } + +#[tokio::test] +async fn test_parallelism_exceed_virtual_node_max_create() -> Result<()> { + let vnode_max = VirtualNode::COUNT; + let mut configuration = Configuration::for_auto_parallelism( + MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE, + true, + ); + + configuration.compute_nodes = 1; + configuration.compute_node_cores = vnode_max + 1; + let mut cluster = Cluster::start(configuration).await?; + + sleep(Duration::from_secs( + MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE * 2, + )) + .await; + + let mut session = cluster.start_session(); + session.run("create table t(v int)").await?; + session + .run("select parallelism from rw_streaming_parallelism where name = 't'") + .await? + .assert_result_eq("ADAPTIVE"); + + session + .run("select distinct parallelism from rw_fragment_parallelism where name = 't'") + .await? + .assert_result_eq(format!("{}", vnode_max)); + + Ok(()) +} + +#[tokio::test] +async fn test_parallelism_exceed_virtual_node_max_alter_fixed() -> Result<()> { + let vnode_max = VirtualNode::COUNT; + let mut configuration = Configuration::for_scale(); + configuration.compute_nodes = 1; + configuration.compute_node_cores = vnode_max + 100; + let mut cluster = Cluster::start(configuration).await?; + let mut session = cluster.start_session(); + session.run("set streaming_parallelism = 1").await?; + session.run("create table t(v int)").await?; + session + .run("select parallelism from rw_streaming_parallelism where name = 't'") + .await? + .assert_result_eq("FIXED(1)"); + + session + .run(format!("alter table t set parallelism = {}", vnode_max + 1)) + .await?; + session + .run("select parallelism from rw_streaming_parallelism where name = 't'") + .await? + .assert_result_eq(format!("FIXED({})", vnode_max)); + Ok(()) +} + +#[tokio::test] +async fn test_parallelism_exceed_virtual_node_max_alter_adaptive() -> Result<()> { + let vnode_max = VirtualNode::COUNT; + let mut configuration = Configuration::for_scale(); + configuration.compute_nodes = 1; + configuration.compute_node_cores = vnode_max + 100; + let mut cluster = Cluster::start(configuration).await?; + let mut session = cluster.start_session(); + session.run("set streaming_parallelism = 1").await?; + session.run("create table t(v int)").await?; + session + .run("select parallelism from rw_streaming_parallelism where name = 't'") + .await? + .assert_result_eq("FIXED(1)"); + + session + .run("alter table t set parallelism = adaptive") + .await?; + session + .run("select parallelism from rw_streaming_parallelism where name = 't'") + .await? + .assert_result_eq("ADAPTIVE"); + + session + .run("select distinct parallelism from rw_fragment_parallelism where name = 't'") + .await? + .assert_result_eq(format!("{}", vnode_max)); + + Ok(()) +} From f1fd63ed1470fb076971df43deb1f85af5027a7a Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Mon, 19 Aug 2024 16:30:18 +0800 Subject: [PATCH 17/26] feat(frontend): support single phase approx percentile in batch (#18083) --- .../aggregate/shuffle_approx_percentile.slt | 33 +++++---- ..._approx_percentile_merge_stateless_agg.slt | 39 ++++++++++ .../tests/testdata/input/agg.yaml | 17 +++++ .../tests/testdata/output/agg.yaml | 71 +++++++++++++++++++ .../optimizer/plan_node/batch_simple_agg.rs | 11 ++- 5 files changed, 157 insertions(+), 14 deletions(-) diff --git a/e2e_test/streaming/aggregate/shuffle_approx_percentile.slt b/e2e_test/streaming/aggregate/shuffle_approx_percentile.slt index efc377f8aed4..3f2c400b91f0 100644 --- a/e2e_test/streaming/aggregate/shuffle_approx_percentile.slt +++ b/e2e_test/streaming/aggregate/shuffle_approx_percentile.slt @@ -58,19 +58,6 @@ select * from m1; ---- -982.5779489474152 -804.4614206837127 0 804.4614206837127 982.5779489474152 -# Test state encode / decode -onlyif can-use-recover -statement ok -recover; - -onlyif can-use-recover -sleep 10s - -query I -select * from m1; ----- --982.5779489474152 -804.4614206837127 0 804.4614206837127 982.5779489474152 - # Test 0 bool { - self.core.can_two_phase_agg() && self.two_phase_agg_enabled() + self.core.can_two_phase_agg() + && self + .core + // Ban two phase approx percentile. + .agg_calls + .iter() + .map(|agg_call| &agg_call.agg_kind) + .all(|agg_kind| !matches!(agg_kind, AggKind::Builtin(PbAggKind::ApproxPercentile))) + && self.two_phase_agg_enabled() } } From 7f80900147bc3da2db31cf386bb25a1b86cbefc0 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 19 Aug 2024 17:01:06 +0800 Subject: [PATCH 18/26] feat(license): add limit for total cpu cores in the cluster (#18022) Signed-off-by: Bugen Zhao --- src/license/src/cpu.rs | 99 ++++++++++++++++++++++++++++++ src/license/src/lib.rs | 1 + src/license/src/manager.rs | 10 ++- src/meta/src/controller/cluster.rs | 47 +++++++++++++- src/meta/src/manager/cluster.rs | 33 +++++++++- 5 files changed, 186 insertions(+), 4 deletions(-) create mode 100644 src/license/src/cpu.rs diff --git a/src/license/src/cpu.rs b/src/license/src/cpu.rs new file mode 100644 index 000000000000..b2c23733ce75 --- /dev/null +++ b/src/license/src/cpu.rs @@ -0,0 +1,99 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::num::NonZeroU64; + +use thiserror::Error; + +use crate::{LicenseKeyError, LicenseManager}; + +/// The error type for CPU core limit exceeded as per the license key. +#[derive(Debug, Clone, Error)] +#[error("invalid license key")] +pub enum CpuCoreLimitExceeded { + #[error("cannot check CPU core limit due to license key error")] + LicenseKeyError(#[from] LicenseKeyError), + + #[error( + "CPU core limit exceeded as per the license key, \ + requesting {actual} while the maximum allowed is {limit}" + )] + Exceeded { limit: NonZeroU64, actual: u64 }, +} + +impl LicenseManager { + /// Check if the given CPU core count exceeds the limit as per the license key. + pub fn check_cpu_core_limit(&self, cpu_core_count: u64) -> Result<(), CpuCoreLimitExceeded> { + let license = self.license()?; + + match license.cpu_core_limit { + Some(limit) if cpu_core_count > limit.get() => Err(CpuCoreLimitExceeded::Exceeded { + limit, + actual: cpu_core_count, + }), + _ => Ok(()), + } + } +} + +// Tests below only work in debug mode. +#[cfg(debug_assertions)] +#[cfg(test)] +mod tests { + use expect_test::expect; + use thiserror_ext::AsReport as _; + + use super::*; + use crate::{LicenseKey, TEST_PAID_LICENSE_KEY_CONTENT}; + + fn do_test(key: &str, cpu_core_count: u64, expect: expect_test::Expect) { + let manager = LicenseManager::new(); + manager.refresh(LicenseKey(key)); + + match manager.check_cpu_core_limit(cpu_core_count) { + Ok(_) => expect.assert_eq("ok"), + Err(error) => expect.assert_eq(&error.to_report_string()), + } + } + + #[test] + fn test_no_limit() { + do_test(TEST_PAID_LICENSE_KEY_CONTENT, 114514, expect!["ok"]); + } + + #[test] + fn test_no_license_key_no_limit() { + do_test("", 114514, expect!["ok"]); + } + + #[test] + fn test_invalid_license_key() { + const KEY: &str = "invalid"; + + do_test(KEY, 0, expect!["cannot check CPU core limit due to license key error: invalid license key: InvalidToken"]); + do_test(KEY, 114514, expect!["cannot check CPU core limit due to license key error: invalid license key: InvalidToken"]); + } + + #[test] + fn test_limit() { + const KEY: &str = + "eyJhbGciOiJSUzUxMiIsInR5cCI6IkpXVCJ9.\ + eyJzdWIiOiJmcmVlLXRlc3QtMzIiLCJpc3MiOiJwcm9kLnJpc2luZ3dhdmUuY29tIiwidGllciI6ImZyZWUiLCJleHAiOjE4NTI1NTk5OTksImlhdCI6MTcyMzcwMTk5NCwiY3B1X2NvcmVfbGltaXQiOjMyfQ.\ + rsATtzlduLUkGQeXkOROtyGUpafdDhi18iKdYAzAldWQuO9KevNcnD8a6geCShZSGte65bI7oYtv7GHx8i66ge3B1SVsgGgYr10ebphPUNUQenYoN0mpD4Wn0prPStOgANzYZOI2ntMGAaeWStji1x67_iho6r0W9r6RX3kMvzFSbiObSIfvTdrMULeg-xeHc3bT_ErRhaXq7MAa2Oiq3lcK2sNgEvc9KYSP9YbhSik9CBkc8lcyeVoc48SSWEaBU-c8-Ge0fzjgWHI9KIsUV5Ihe66KEfs0PqdRoSWbgskYGzA3o8wHIbtJbJiPzra373kkFH9MGY0HOsw9QeJLGQ"; + + do_test(KEY, 31, expect!["ok"]); + do_test(KEY, 32, expect!["ok"]); + do_test(KEY, 33, expect!["CPU core limit exceeded as per the license key, requesting 33 while the maximum allowed is 32"]); + } +} diff --git a/src/license/src/lib.rs b/src/license/src/lib.rs index bdcac9044104..e2a327578009 100644 --- a/src/license/src/lib.rs +++ b/src/license/src/lib.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod cpu; mod feature; mod key; mod manager; diff --git a/src/license/src/manager.rs b/src/license/src/manager.rs index cac51105358a..5c1bc298388d 100644 --- a/src/license/src/manager.rs +++ b/src/license/src/manager.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::num::NonZeroU64; use std::sync::{LazyLock, RwLock}; use jsonwebtoken::{Algorithm, DecodingKey, Validation}; @@ -76,6 +77,9 @@ pub(super) struct License { /// Tier of the license. pub tier: Tier, + /// Maximum number of compute-node CPU cores allowed to use. Typically used for the paid tier. + pub cpu_core_limit: Option, + /// Expiration time in seconds since UNIX epoch. /// /// See . @@ -91,6 +95,7 @@ impl Default for License { sub: "default".to_owned(), tier: Tier::Free, iss: Issuer::Prod, + cpu_core_limit: None, exp: u64::MAX, } } @@ -117,7 +122,7 @@ static PUBLIC_KEY: LazyLock = LazyLock::new(|| { impl LicenseManager { /// Create a new license manager with the default license. - fn new() -> Self { + pub(crate) fn new() -> Self { Self { inner: RwLock::new(Inner { license: Ok(License::default()), @@ -208,6 +213,7 @@ mod tests { sub: "rw-test", iss: Test, tier: Paid, + cpu_core_limit: None, exp: 9999999999, } "#]], @@ -228,6 +234,7 @@ mod tests { sub: "rw-test", iss: Test, tier: Free, + cpu_core_limit: None, exp: 9999999999, } "#]], @@ -244,6 +251,7 @@ mod tests { sub: "default", iss: Prod, tier: Free, + cpu_core_limit: None, exp: 18446744073709551615, } "#]], diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index 6a7ca826f160..10d4a947e2a9 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -26,6 +26,7 @@ use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::RW_VERSION; use risingwave_hummock_sdk::HummockSstableObjectId; +use risingwave_license::LicenseManager; use risingwave_meta_model_v2::prelude::{Worker, WorkerProperty}; use risingwave_meta_model_v2::worker::{WorkerStatus, WorkerType}; use risingwave_meta_model_v2::{worker, worker_property, TransactionId, WorkerId}; @@ -38,8 +39,8 @@ use risingwave_pb::meta::update_worker_node_schedulability_request::Schedulabili use sea_orm::prelude::Expr; use sea_orm::ActiveValue::Set; use sea_orm::{ - ActiveModelTrait, ColumnTrait, DatabaseConnection, EntityTrait, QueryFilter, QuerySelect, - TransactionTrait, + ActiveModelTrait, ColumnTrait, DatabaseConnection, DatabaseTransaction, EntityTrait, + QueryFilter, QuerySelect, TransactionTrait, }; use thiserror_ext::AsReport; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver}; @@ -581,6 +582,43 @@ impl ClusterControllerInner { } } + /// Check if the total CPU cores in the cluster exceed the license limit, after counting the + /// newly joined compute node. + pub async fn check_cpu_core_limit_on_newly_joined_compute_node( + &self, + txn: &DatabaseTransaction, + host_address: &HostAddress, + resource: &PbResource, + ) -> MetaResult<()> { + let this = resource.total_cpu_cores; + + let other_worker_ids: Vec = Worker::find() + .filter( + (worker::Column::Host + .eq(host_address.host.clone()) + .and(worker::Column::Port.eq(host_address.port))) + .not() + .and(worker::Column::WorkerType.eq(WorkerType::ComputeNode as i32)), + ) + .select_only() + .column(worker::Column::WorkerId) + .into_tuple() + .all(txn) + .await?; + + let others = other_worker_ids + .into_iter() + .flat_map(|id| self.worker_extra_info.get(&id)) + .flat_map(|info| info.resource.as_ref().map(|r| r.total_cpu_cores)) + .sum::(); + + LicenseManager::get() + .check_cpu_core_limit(this + others) + .map_err(anyhow::Error::from)?; + + Ok(()) + } + pub async fn add_worker( &mut self, r#type: PbWorkerType, @@ -591,6 +629,11 @@ impl ClusterControllerInner { ) -> MetaResult { let txn = self.db.begin().await?; + if let PbWorkerType::ComputeNode = r#type { + self.check_cpu_core_limit_on_newly_joined_compute_node(&txn, &host_address, &resource) + .await?; + } + let worker = Worker::find() .filter( worker::Column::Host diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 93e50dec3706..d5c12c70a0b9 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -24,7 +24,8 @@ use risingwave_common::util::addr::HostAddr; use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::RW_VERSION; -use risingwave_pb::common::worker_node::{Property, State}; +use risingwave_license::LicenseManager; +use risingwave_pb::common::worker_node::{Property, Resource, State}; use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; use risingwave_pb::meta::add_worker_node_request::Property as AddNodeProperty; use risingwave_pb::meta::heartbeat_request; @@ -114,6 +115,13 @@ impl ClusterManager { let mut property = self.parse_property(r#type, property); let mut core = self.core.write().await; + if let WorkerType::ComputeNode = r#type { + core.check_cpu_core_limit_on_newly_joined_compute_node( + host_address.clone(), + &resource, + )?; + } + if let Some(worker) = core.get_worker_by_host_mut(host_address.clone()) { tracing::info!("worker {} re-joined the cluster", worker.worker_id()); worker.update_resource(Some(resource)); @@ -631,6 +639,29 @@ impl ClusterManagerCore { .map(|(_, worker)| worker.clone()) } + /// Check if the total CPU cores in the cluster exceed the license limit, after counting the + /// newly joined compute node. + pub fn check_cpu_core_limit_on_newly_joined_compute_node( + &self, + host_address: HostAddress, + resource: &Resource, + ) -> MetaResult<()> { + let this_key = WorkerKey(host_address); + + let this = resource.total_cpu_cores; + let others = (self.workers.iter()) + .filter(|(k, _v)| k != &&this_key) + .filter(|(_k, v)| v.worker_node.r#type == WorkerType::ComputeNode as i32) + .flat_map(|(_k, v)| v.resource.as_ref().map(|r| r.total_cpu_cores)) + .sum::(); + + LicenseManager::get() + .check_cpu_core_limit(this + others) + .map_err(anyhow::Error::from)?; + + Ok(()) + } + fn add_worker_node(&mut self, worker: Worker) { if let Some(transactional_id) = worker.worker_node.transactional_id { self.available_transactional_ids From da74340e3ed5a1d7cbf1657343612cf74956e610 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Mon, 19 Aug 2024 17:05:03 +0800 Subject: [PATCH 19/26] fix(parser): fix unparse of include columns (#18098) --- src/sqlparser/src/ast/mod.rs | 2 +- src/sqlparser/tests/testdata/create.yaml | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 73c174769eb4..eeb985967971 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1840,7 +1840,7 @@ impl fmt::Display for Statement { if !include_column_options.is_empty() { // (Ident, Option) write!(f, "{}", display_comma_separated( include_column_options.iter().map(|option_item: &IncludeOptionItem| { - format!("INCLUDE {}{}{}", + format!(" INCLUDE {}{}{}", option_item.column_type, if let Some(inner_field) = &option_item.inner_field { format!(" {}", inner_field) diff --git a/src/sqlparser/tests/testdata/create.yaml b/src/sqlparser/tests/testdata/create.yaml index 317130b79d5e..13f5e2dad5bf 100644 --- a/src/sqlparser/tests/testdata/create.yaml +++ b/src/sqlparser/tests/testdata/create.yaml @@ -79,6 +79,8 @@ ^ - input: CREATE TABLE T ("FULL" INT) ON CONFLICT DO UPDATE IF NOT NULL formatted_sql: CREATE TABLE T ("FULL" INT) ON CONFLICT DO UPDATE IF NOT NULL +- input: CREATE TABLE t (a int, b int, ts timestamptz as proctime(), primary key (a)) ON CONFLICT DO UPDATE IF NOT NULL INCLUDE timestamp AS ts_col WITH (connector = 'kafka', properties.bootstrap.server = '127.0.0.1:9092', topic = 'test_topic') FORMAT PLAIN ENCODE JSON + formatted_sql: CREATE TABLE t (a INT, b INT, ts timestamptz AS proctime(), PRIMARY KEY (a)) ON CONFLICT DO UPDATE IF NOT NULL INCLUDE timestamp AS ts_col WITH (connector = 'kafka', properties.bootstrap.server = '127.0.0.1:9092', topic = 'test_topic') FORMAT PLAIN ENCODE JSON - input: CREATE USER user WITH SUPERUSER CREATEDB PASSWORD 'password' formatted_sql: CREATE USER user WITH SUPERUSER CREATEDB PASSWORD 'password' - input: CREATE SINK snk From a9bfcd27e4e748d476ead2d606037049cd3cff44 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Mon, 19 Aug 2024 17:14:31 +0800 Subject: [PATCH 20/26] fix(meta): adapt time travel for partial checkpoint (#18093) --- src/meta/src/hummock/manager/time_travel.rs | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/src/meta/src/hummock/manager/time_travel.rs b/src/meta/src/hummock/manager/time_travel.rs index 61bcdfe7e8de..142c48e9bc0f 100644 --- a/src/meta/src/hummock/manager/time_travel.rs +++ b/src/meta/src/hummock/manager/time_travel.rs @@ -395,9 +395,13 @@ impl HummockManager { version_id: Set(version_id.try_into().unwrap()), }; hummock_epoch_to_version::Entity::insert(m) + .on_conflict( + OnConflict::column(hummock_epoch_to_version::Column::Epoch) + .update_columns([hummock_epoch_to_version::Column::VersionId]) + .to_owned(), + ) .exec(txn) .await?; - let mut version_sst_ids = None; let select_groups = group_parents .iter() @@ -483,14 +487,8 @@ fn replay_archive( deltas: impl Iterator, ) -> HummockVersion { let mut last_version = HummockVersion::from_persisted_protobuf(&version); - let mut mce = last_version.visible_table_committed_epoch(); for d in deltas { let d = HummockVersionDelta::from_persisted_protobuf(&d); - assert!( - d.visible_table_committed_epoch() > mce, - "time travel expects delta from commit_epoch only" - ); - mce = d.visible_table_committed_epoch(); // Need to work around the assertion in `apply_version_delta`. // Because compaction deltas are not included in time travel archive. while last_version.id < d.prev_id { From f98904d6f06117827883f2c9823adb0aa3eb819c Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 19 Aug 2024 17:30:42 +0800 Subject: [PATCH 21/26] feat: support more map functions (#18073) --- e2e_test/batch/types/map.slt.part | 139 ++++++++++++--- proto/expr.proto | 9 + src/common/src/array/list_array.rs | 4 +- src/common/src/array/map_array.rs | 92 +++++++++- src/common/src/array/struct_array.rs | 9 + src/common/src/test_utils/rand_array.rs | 2 +- src/common/src/types/map_type.rs | 43 +++-- src/common/src/types/mod.rs | 8 +- src/common/src/util/value_encoding/mod.rs | 2 +- src/connector/codec/src/decoder/avro/mod.rs | 2 +- src/expr/impl/src/scalar/array.rs | 160 ++++++++++++++++-- src/expr/impl/src/scalar/cast.rs | 2 +- .../binder/expr/function/builtin_scalar.rs | 9 + src/frontend/src/binder/expr/mod.rs | 2 +- src/frontend/src/binder/expr/value.rs | 4 +- src/frontend/src/expr/mod.rs | 15 +- src/frontend/src/expr/pure.rs | 11 +- src/frontend/src/expr/type_inference/func.rs | 36 +++- .../src/optimizer/plan_expr_visitor/strong.rs | 9 + 19 files changed, 482 insertions(+), 76 deletions(-) diff --git a/e2e_test/batch/types/map.slt.part b/e2e_test/batch/types/map.slt.part index 5f68bcad2274..b4b4be7e5cba 100644 --- a/e2e_test/batch/types/map.slt.part +++ b/e2e_test/batch/types/map.slt.part @@ -8,66 +8,66 @@ create table t (m map (float, float)); db error: ERROR: Failed to run the query Caused by: - invalid map key type: double precision + Bind error: invalid map key type: double precision query error -select map_from_entries(array[1.0,2.0,3.0], array[1,2,3]); +select map_from_key_values(array[1.0,2.0,3.0], array[1,2,3]); ---- db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): - 1: Failed to bind expression: map_from_entries(ARRAY[1.0, 2.0, 3.0], ARRAY[1, 2, 3]) + 1: Failed to bind expression: map_from_key_values(ARRAY[1.0, 2.0, 3.0], ARRAY[1, 2, 3]) 2: Expr error 3: invalid map key type: numeric query error -select map_from_entries(array[1,1,3], array[1,2,3]); +select map_from_key_values(array[1,1,3], array[1,2,3]); ---- db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map_from_entries('{1,1,3}', '{1,2,3}')` + 2: error while evaluating expression `map_from_key_values('{1,1,3}', '{1,2,3}')` 3: map keys must be unique query ? -select map_from_entries(array[1,2,3], array[1,null,3]); +select map_from_key_values(array[1,2,3], array[1,null,3]); ---- {1:1,2:NULL,3:3} query error -select map_from_entries(array[1,null,3], array[1,2,3]); +select map_from_key_values(array[1,null,3], array[1,2,3]); ---- db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map_from_entries('{1,NULL,3}', '{1,2,3}')` + 2: error while evaluating expression `map_from_key_values('{1,NULL,3}', '{1,2,3}')` 3: map keys must not be NULL query error -select map_from_entries(array[1,3], array[1,2,3]); +select map_from_key_values(array[1,3], array[1,2,3]); ---- db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map_from_entries('{1,3}', '{1,2,3}')` + 2: error while evaluating expression `map_from_key_values('{1,3}', '{1,2,3}')` 3: map keys and values have different length query error -select map_from_entries(array[1,2], array[1,2]) = map_from_entries(array[2,1], array[2,1]); +select map_from_key_values(array[1,2], array[1,2]) = map_from_key_values(array[2,1], array[2,1]); ---- db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): - 1: Failed to bind expression: map_from_entries(ARRAY[1, 2], ARRAY[1, 2]) = map_from_entries(ARRAY[2, 1], ARRAY[2, 1]) + 1: Failed to bind expression: map_from_key_values(ARRAY[1, 2], ARRAY[1, 2]) = map_from_key_values(ARRAY[2, 1], ARRAY[2, 1]) 2: function equal(map(integer,integer), map(integer,integer)) does not exist @@ -83,32 +83,32 @@ create table t ( statement ok insert into t values ( - map_from_entries(array['a','b','c'], array[1.0,2.0,3.0]::float[]), - map_from_entries(array[1,2,3], array[true,false,true]), - map_from_entries(array['a','b'], + map_from_key_values(array['a','b','c'], array[1.0,2.0,3.0]::float[]), + map_from_key_values(array[1,2,3], array[true,false,true]), + map_from_key_values(array['a','b'], array[ - map_from_entries(array['a1'], array['a2']), - map_from_entries(array['b1'], array['b2']) + map_from_key_values(array['a1'], array['a2']), + map_from_key_values(array['b1'], array['b2']) ] ), array[ - map_from_entries(array['a','b','c'], array[1,2,3]), - map_from_entries(array['d','e','f'], array[4,5,6]) + map_from_key_values(array['a','b','c'], array[1,2,3]), + map_from_key_values(array['d','e','f'], array[4,5,6]) ], row( - map_from_entries(array['a','b','c'], array[row(1),row(2),row(3)]::struct[]) + map_from_key_values(array['a','b','c'], array[row(1),row(2),row(3)]::struct[]) ) ); # cast(map(character varying,integer)) -> map(character varying,double precision) query ? -select map_from_entries(array['a','b','c'], array[1,2,3])::map(varchar,float); +select map_from_key_values(array['a','b','c'], array[1,2,3])::map(varchar,float); ---- {a:1,b:2,c:3} statement ok -insert into t(m1) values (map_from_entries(array['a','b','c'], array[1,2,3])); +insert into t(m1) values (map_from_key_values(array['a','b','c'], array[1,2,3])); query ????? rowsort select * from t; @@ -144,7 +144,7 @@ db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map_from_entries('{a,a}', '{1,2}')` + 2: error while evaluating expression `map_from_key_values('{a,a}', '{1,2}')` 3: map keys must be unique @@ -165,3 +165,96 @@ select MAP{1:'a',2:'b'}::MAP(VARCHAR,VARCHAR) ---- {} {1:a,2:b} + +query error +select map_from_entries(array[]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(ARRAY[]) + 2: Bind error: cannot determine type of empty array +HINT: Explicitly cast to the desired type, for example ARRAY[]::integer[]. + + +query error +select map_from_entries(array[]::int[]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(CAST(ARRAY[] AS INT[])) + 2: Expr error + 3: invalid map entries type, expected struct, got: integer + + +query error +select map_from_entries(array[]::struct[]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(CAST(ARRAY[] AS STRUCT[])) + 2: Expr error + 3: invalid map key type: double precision + + +query ? +select map_from_entries(array[]::struct[]); +---- +{} + + +query ? +select map_from_entries(array[row('a',1), row('b',2), row('c',3)]); +---- +{a:1,b:2,c:3} + + +query error +select map_from_entries(array[row('a',1), row('a',2), row('c',3)]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Expr error + 2: error while evaluating expression `map_from_entries('{"(a,1)","(a,2)","(c,3)"}')` + 3: map keys must be unique + + +query error +select map_from_entries(array[row('a',1,2)]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(ARRAY[ROW('a', 1, 2)]) + 2: Expr error + 3: the underlying struct for map must have exactly two fields, got: StructType { field_names: [], field_types: [Varchar, Int32, Int32] } + + +query error +select map_from_entries(array[row(1.0,1)]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(ARRAY[ROW(1.0, 1)]) + 2: Expr error + 3: invalid map key type: numeric + + +query error +select map_from_entries(null); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(NULL) + 2: Bind error: Cannot implicitly cast 'null:Varchar' to polymorphic type AnyArray + + +query ? +select map_from_entries(null::struct[]); +---- +NULL diff --git a/proto/expr.proto b/proto/expr.proto index 0f543d3514e3..e5b5fb73ba8f 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -286,6 +286,15 @@ message ExprNode { // Map functions MAP_FROM_ENTRIES = 700; MAP_ACCESS = 701; + MAP_KEYS = 702; + MAP_VALUES = 703; + MAP_ENTRIES = 704; + MAP_FROM_KEY_VALUES = 705; + MAP_LENGTH = 706; + MAP_CONTAINS = 707; + MAP_CAT = 708; + MAP_INSERT = 709; + MAP_DELETE = 710; // Non-pure functions below (> 1000) // ------------------------ diff --git a/src/common/src/array/list_array.rs b/src/common/src/array/list_array.rs index c30229852c0a..745b1f6bbab0 100644 --- a/src/common/src/array/list_array.rs +++ b/src/common/src/array/list_array.rs @@ -372,11 +372,11 @@ impl ListValue { /// Creates a new `ListValue` from an iterator of `Datum`. pub fn from_datum_iter( - datatype: &DataType, + elem_datatype: &DataType, iter: impl IntoIterator, ) -> Self { let iter = iter.into_iter(); - let mut builder = datatype.create_array_builder(iter.size_hint().0); + let mut builder = elem_datatype.create_array_builder(iter.size_hint().0); for datum in iter { builder.append(datum); } diff --git a/src/common/src/array/map_array.rs b/src/common/src/array/map_array.rs index f0904211f5ed..2f0da9bbf816 100644 --- a/src/common/src/array/map_array.rs +++ b/src/common/src/array/map_array.rs @@ -24,6 +24,7 @@ use serde::Serializer; use super::{ Array, ArrayBuilder, ArrayImpl, ArrayResult, DatumRef, DefaultOrdered, ListArray, ListArrayBuilder, ListRef, ListValue, MapType, ScalarRef, ScalarRefImpl, StructArray, + StructRef, }; use crate::bitmap::Bitmap; use crate::types::{DataType, Scalar, ToText}; @@ -162,7 +163,7 @@ impl Array for MapArray { fn data_type(&self) -> DataType { let list_value_type = self.inner.values().data_type(); - DataType::Map(MapType::from_list_entries(list_value_type)) + DataType::Map(MapType::from_entries(list_value_type)) } } @@ -193,7 +194,10 @@ pub use scalar::{MapRef, MapValue}; /// We only check the invariants in the constructors. /// After they are constructed, we assume the invariants holds. mod scalar { + use std::collections::HashSet; + use super::*; + use crate::array::{Datum, ScalarImpl, StructValue}; /// Refer to [`MapArray`] for the invariants of a map value. #[derive(Clone, Eq, EstimateSize)] @@ -221,20 +225,33 @@ mod scalar { /// # Panics /// Panics if [map invariants](`super::MapArray`) are violated. - pub fn from_list_entries(list: ListValue) -> Self { + pub fn from_entries(entries: ListValue) -> Self { + Self::try_from_entries(entries).unwrap() + } + + /// Returns error if [map invariants](`super::MapArray`) are violated. + pub fn try_from_entries(entries: ListValue) -> Result { // validates list type is valid - _ = MapType::from_list_entries(list.data_type()); - // TODO: validate the values is valid - MapValue(list) + let _ = MapType::try_from_entries(entries.data_type())?; + let mut keys = HashSet::with_capacity(entries.len()); + let struct_array = entries.into_array(); + for key in struct_array.as_struct().field_at(0).iter() { + let Some(key) = key else { + return Err("map keys must not be NULL".to_string()); + }; + if !keys.insert(key) { + return Err("map keys must be unique".to_string()); + } + } + Ok(MapValue(ListValue::new(struct_array))) } - /// # Panics - /// Panics if [map invariants](`super::MapArray`) are violated. + /// Returns error if [map invariants](`super::MapArray`) are violated. pub fn try_from_kv(key: ListValue, value: ListValue) -> Result { if key.len() != value.len() { return Err("map keys and values have different length".to_string()); } - let unique_keys = key.iter().unique().collect_vec(); + let unique_keys: HashSet<_> = key.iter().unique().collect(); if unique_keys.len() != key.len() { return Err("map keys must be unique".to_string()); } @@ -252,6 +269,46 @@ mod scalar { ); Ok(MapValue(ListValue::new(struct_array.into()))) } + + /// # Panics + /// Panics if `m1` and `m2` have different types. + pub fn concat(m1: MapRef<'_>, m2: MapRef<'_>) -> Self { + debug_assert_eq!(m1.inner().data_type(), m2.inner().data_type()); + let m2_keys = m2.keys(); + let l = ListValue::from_datum_iter( + &m1.inner().data_type(), + m1.iter_struct() + .filter(|s| !m2_keys.contains(&s.field_at(0).expect("map key is not null"))) + .chain(m2.iter_struct()) + .map(|s| Some(ScalarRefImpl::Struct(s))), + ); + Self::from_entries(l) + } + + pub fn insert(m: MapRef<'_>, key: ScalarImpl, value: Datum) -> Self { + let l = ListValue::from_datum_iter( + &m.inner().data_type(), + m.iter_struct() + .filter(|s| { + key.as_scalar_ref_impl() != s.field_at(0).expect("map key is not null") + }) + .chain(std::iter::once( + StructValue::new(vec![Some(key.clone()), value]).as_scalar_ref(), + )) + .map(|s| Some(ScalarRefImpl::Struct(s))), + ); + Self::from_entries(l) + } + + pub fn delete(m: MapRef<'_>, key: ScalarRefImpl<'_>) -> Self { + let l = ListValue::from_datum_iter( + &m.inner().data_type(), + m.iter_struct() + .filter(|s| key != s.field_at(0).expect("map key is not null")) + .map(|s| Some(ScalarRefImpl::Struct(s))), + ); + Self::from_entries(l) + } } impl<'a> MapRef<'a> { @@ -272,6 +329,14 @@ mod scalar { pub fn into_kv(self) -> (ListRef<'a>, ListRef<'a>) { self.0.as_map_kv() } + + pub fn keys(&self) -> HashSet> { + self.iter().map(|(k, _v)| k).collect() + } + + pub fn to_owned(self) -> MapValue { + MapValue(self.0.to_owned()) + } } impl Scalar for MapValue { @@ -379,6 +444,15 @@ impl<'a> MapRef<'a> { }) } + pub fn iter_struct( + self, + ) -> impl DoubleEndedIterator + ExactSizeIterator> + 'a { + self.inner().iter().map(|list_elem| { + let list_elem = list_elem.expect("the list element in map should not be null"); + list_elem.into_struct() + }) + } + pub fn iter_sorted( self, ) -> impl DoubleEndedIterator + ExactSizeIterator, DatumRef<'a>)> + 'a @@ -411,7 +485,7 @@ impl MapValue { deserializer: &mut memcomparable::Deserializer, ) -> memcomparable::Result { let list = ListValue::memcmp_deserialize(&datatype.clone().into_struct(), deserializer)?; - Ok(Self::from_list_entries(list)) + Ok(Self::from_entries(list)) } } diff --git a/src/common/src/array/struct_array.rs b/src/common/src/array/struct_array.rs index 9c3bd2365381..ebf224f58161 100644 --- a/src/common/src/array/struct_array.rs +++ b/src/common/src/array/struct_array.rs @@ -393,6 +393,15 @@ impl<'a> StructRef<'a> { iter_fields_ref!(self, it, { Either::Left(it) }, { Either::Right(it) }) } + /// # Panics + /// Panics if the index is out of bounds. + pub fn field_at(&self, i: usize) -> DatumRef<'a> { + match self { + StructRef::Indexed { arr, idx } => arr.field_at(i).value_at(*idx), + StructRef::ValueRef { val } => val.fields[i].to_datum_ref(), + } + } + pub fn memcmp_serialize( self, serializer: &mut memcomparable::Serializer, diff --git a/src/common/src/test_utils/rand_array.rs b/src/common/src/test_utils/rand_array.rs index f201b4de3384..33cf42bc403e 100644 --- a/src/common/src/test_utils/rand_array.rs +++ b/src/common/src/test_utils/rand_array.rs @@ -154,7 +154,7 @@ impl RandValue for ListValue { impl RandValue for MapValue { fn rand_value(_rand: &mut R) -> Self { // dummy value - MapValue::from_list_entries(ListValue::empty(&DataType::Struct( + MapValue::from_entries(ListValue::empty(&DataType::Struct( MapType::struct_type_for_map(DataType::Varchar, DataType::Varchar), ))) } diff --git a/src/common/src/types/map_type.rs b/src/common/src/types/map_type.rs index 4d9ec3dc5f14..e0dae8d9bc10 100644 --- a/src/common/src/types/map_type.rs +++ b/src/common/src/types/map_type.rs @@ -36,26 +36,37 @@ impl MapType { Self(Box::new((key, value))) } - pub fn try_from_kv(key: DataType, value: DataType) -> Result { + pub fn try_from_kv(key: DataType, value: DataType) -> Result { Self::check_key_type_valid(&key)?; Ok(Self(Box::new((key, value)))) } + pub fn try_from_entries(list_entries_type: DataType) -> Result { + match list_entries_type { + DataType::Struct(s) => { + let Some((k, v)) = s.iter().collect_tuple() else { + return Err(format!( + "the underlying struct for map must have exactly two fields, got: {s:?}" + )); + }; + // the field names are not strictly enforced + // Currently this panics for SELECT * FROM t + // if cfg!(debug_assertions) { + // itertools::assert_equal(struct_type.names(), ["key", "value"]); + // } + Self::try_from_kv(k.1.clone(), v.1.clone()) + } + _ => Err(format!( + "invalid map entries type, expected struct, got: {list_entries_type}" + )), + } + } + /// # Panics /// Panics if the key type is not valid for a map, or the /// entries type is not a valid struct type. - pub fn from_list_entries(list_entries_type: DataType) -> Self { - let struct_type = list_entries_type.as_struct(); - let (k, v) = struct_type - .iter() - .collect_tuple() - .expect("the underlying struct for map must have exactly two fields"); - // the field names are not strictly enforced - // Currently this panics for SELECT * FROM t - // if cfg!(debug_assertions) { - // itertools::assert_equal(struct_type.names(), ["key", "value"]); - // } - Self::from_kv(k.1.clone(), v.1.clone()) + pub fn from_entries(list_entries_type: DataType) -> Self { + Self::try_from_entries(list_entries_type).unwrap() } /// # Panics @@ -89,7 +100,7 @@ impl MapType { /// /// Note that this isn't definitive. /// Just be conservative at the beginning, but not too restrictive (like only allowing strings). - pub fn check_key_type_valid(data_type: &DataType) -> anyhow::Result<()> { + pub fn check_key_type_valid(data_type: &DataType) -> Result<(), String> { let ok = match data_type { DataType::Int16 | DataType::Int32 | DataType::Int64 => true, DataType::Varchar => true, @@ -111,7 +122,7 @@ impl MapType { | DataType::Map(_) => false, }; if !ok { - Err(anyhow::anyhow!("invalid map key type: {data_type}")) + Err(format!("invalid map key type: {data_type}")) } else { Ok(()) } @@ -128,7 +139,7 @@ impl FromStr for MapType { if let Some((key, value)) = s[4..s.len() - 1].split(',').collect_tuple() { let key = key.parse().context("failed to parse map key type")?; let value = value.parse().context("failed to parse map value type")?; - MapType::try_from_kv(key, value) + MapType::try_from_kv(key, value).map_err(|e| anyhow::anyhow!(e)) } else { Err(anyhow::anyhow!("expect map(...,...)")) } diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index b86e70b85d8b..1fe1f3958e33 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -251,7 +251,7 @@ impl From<&PbDataType> for DataType { // Map is physically the same as a list. // So the first (and only) item is the list element type. let list_entries_type: DataType = (&proto.field_type[0]).into(); - DataType::Map(MapType::from_list_entries(list_entries_type)) + DataType::Map(MapType::from_entries(list_entries_type)) } PbTypeName::Int256 => DataType::Int256, } @@ -849,6 +849,12 @@ impl From for ScalarImpl { } } +impl From> for ScalarImpl { + fn from(list: ListRef<'_>) -> Self { + Self::List(list.to_owned_scalar()) + } +} + impl ScalarImpl { /// Creates a scalar from pgwire "BINARY" format. /// diff --git a/src/common/src/util/value_encoding/mod.rs b/src/common/src/util/value_encoding/mod.rs index 3b4167331cb7..3fdb8078fdef 100644 --- a/src/common/src/util/value_encoding/mod.rs +++ b/src/common/src/util/value_encoding/mod.rs @@ -360,7 +360,7 @@ fn deserialize_value(ty: &DataType, data: &mut impl Buf) -> Result { DataType::Map(map_type) => { // FIXME: clone type everytime here is inefficient let list = deserialize_list(&map_type.clone().into_struct(), data)?.into_list(); - ScalarImpl::Map(MapValue::from_list_entries(list)) + ScalarImpl::Map(MapValue::from_entries(list)) } }) } diff --git a/src/connector/codec/src/decoder/avro/mod.rs b/src/connector/codec/src/decoder/avro/mod.rs index 738535ec9410..dc4dae49ca7c 100644 --- a/src/connector/codec/src/decoder/avro/mod.rs +++ b/src/connector/codec/src/decoder/avro/mod.rs @@ -344,7 +344,7 @@ impl<'a> AvroParseOptions<'a> { ); } let list = ListValue::new(builder.finish()); - MapValue::from_list_entries(list).into() + MapValue::from_entries(list).into() } (_expected, _got) => Err(create_error())?, diff --git a/src/expr/impl/src/scalar/array.rs b/src/expr/impl/src/scalar/array.rs index cee7de36c717..d5f53213bf27 100644 --- a/src/expr/impl/src/scalar/array.rs +++ b/src/expr/impl/src/scalar/array.rs @@ -32,8 +32,14 @@ fn row_(row: impl Row) -> StructValue { StructValue::new(row.iter().map(|d| d.to_owned_datum()).collect()) } -fn map_type_infer(args: &[DataType]) -> Result { - let map = MapType::try_from_kv(args[0].as_list().clone(), args[1].as_list().clone())?; +fn map_from_key_values_type_infer(args: &[DataType]) -> Result { + let map = MapType::try_from_kv(args[0].as_list().clone(), args[1].as_list().clone()) + .map_err(ExprError::Custom)?; + Ok(map.into()) +} + +fn map_from_entries_type_infer(args: &[DataType]) -> Result { + let map = MapType::try_from_entries(args[0].as_list().clone()).map_err(ExprError::Custom)?; Ok(map.into()) } @@ -41,62 +47,70 @@ fn map_type_infer(args: &[DataType]) -> Result { /// /// ```slt /// query T -/// select map_from_entries(null::int[], array[1,2,3]); +/// select map_from_key_values(null::int[], array[1,2,3]); /// ---- /// NULL /// /// query T -/// select map_from_entries(array['a','b','c'], array[1,2,3]); +/// select map_from_key_values(array['a','b','c'], array[1,2,3]); /// ---- /// {a:1,b:2,c:3} /// ``` #[function( - "map_from_entries(anyarray, anyarray) -> anymap", - type_infer = "map_type_infer" + "map_from_key_values(anyarray, anyarray) -> anymap", + type_infer = "map_from_key_values_type_infer" )] -fn map_from_entries(key: ListRef<'_>, value: ListRef<'_>) -> Result { +fn map_from_key_values(key: ListRef<'_>, value: ListRef<'_>) -> Result { MapValue::try_from_kv(key.to_owned(), value.to_owned()).map_err(ExprError::Custom) } +#[function( + "map_from_entries(anyarray) -> anymap", + type_infer = "map_from_entries_type_infer" +)] +fn map_from_entries(entries: ListRef<'_>) -> Result { + MapValue::try_from_entries(entries.to_owned()).map_err(ExprError::Custom) +} + /// # Example /// /// ```slt /// query T -/// select map_access(map_from_entries(array[1,2,3], array[100,200,300]), 3); +/// select map_access(map_from_key_values(array[1,2,3], array[100,200,300]), 3); /// ---- /// 300 /// /// query T -/// select map_access(map_from_entries(array[1,2,3], array[100,200,300]), '3'); +/// select map_access(map_from_key_values(array[1,2,3], array[100,200,300]), '3'); /// ---- /// 300 /// /// query error -/// select map_access(map_from_entries(array[1,2,3], array[100,200,300]), 1.0); +/// select map_access(map_from_key_values(array[1,2,3], array[100,200,300]), 1.0); /// ---- /// db error: ERROR: Failed to run the query /// /// Caused by these errors (recent errors listed first): -/// 1: Failed to bind expression: map_access(map_from_entries(ARRAY[1, 2, 3], ARRAY[100, 200, 300]), 1.0) +/// 1: Failed to bind expression: map_access(map_from_key_values(ARRAY[1, 2, 3], ARRAY[100, 200, 300]), 1.0) /// 2: Bind error: Cannot access numeric in map(integer,integer) /// /// /// query T -/// select map_access(map_from_entries(array['a','b','c'], array[1,2,3]), 'a'); +/// select map_access(map_from_key_values(array['a','b','c'], array[1,2,3]), 'a'); /// ---- /// 1 /// /// query T -/// select map_access(map_from_entries(array['a','b','c'], array[1,2,3]), 'd'); +/// select map_access(map_from_key_values(array['a','b','c'], array[1,2,3]), 'd'); /// ---- /// NULL /// /// query T -/// select map_access(map_from_entries(array['a','b','c'], array[1,2,3]), null); +/// select map_access(map_from_key_values(array['a','b','c'], array[1,2,3]), null); /// ---- /// NULL /// ``` -#[function("map_access(anymap, any) -> any")] +#[function("map_access(anymap, any) -> any", type_infer = "unreachable")] fn map_access<'a>( map: MapRef<'a>, key: ScalarRefImpl<'_>, @@ -111,6 +125,122 @@ fn map_access<'a>( } } +/// ```slt +/// query T +/// select +/// map_contains(MAP{1:1}, 1), +/// map_contains(MAP{1:1}, 2), +/// map_contains(MAP{1:1}, NULL::varchar), +/// map_contains(MAP{1:1}, 1.0) +/// ---- +/// t f NULL f +/// ``` +#[function("map_contains(anymap, any) -> boolean")] +fn map_contains(map: MapRef<'_>, key: ScalarRefImpl<'_>) -> Result { + let (keys, _values) = map.into_kv(); + let idx = array_position(keys, Some(key))?; + Ok(idx.is_some()) +} + +/// ```slt +/// query I +/// select +/// map_length(NULL::map(int,int)), +/// map_length(MAP {}::map(int,int)), +/// map_length(MAP {1:1,2:2}::map(int,int)) +/// ---- +/// NULL 0 2 +/// ``` +#[function("map_length(anymap) -> int4")] +fn map_length>(map: MapRef<'_>) -> Result { + map.inner() + .len() + .try_into() + .map_err(|_| ExprError::NumericOverflow) +} + +/// If both `m1` and `m2` have a value with the same key, then the output map contains the value from `m2`. +/// +/// ```slt +/// query T +/// select map_cat(MAP{'a':1,'b':2},null::map(varchar,int)); +/// ---- +/// {a:1,b:2} +/// +/// query T +/// select map_cat(MAP{'a':1,'b':2},MAP{'b':3,'c':4}); +/// ---- +/// {a:1,b:3,c:4} +/// +/// # implicit type cast +/// query T +/// select map_cat(MAP{'a':1,'b':2},MAP{'b':3.0,'c':4.0}); +/// ---- +/// {a:1,b:3.0,c:4.0} +/// ``` +#[function("map_cat(anymap, anymap) -> anymap")] +fn map_cat(m1: Option>, m2: Option>) -> Result, ExprError> { + match (m1, m2) { + (None, None) => Ok(None), + (Some(m), None) | (None, Some(m)) => Ok(Some(m.to_owned())), + (Some(m1), Some(m2)) => Ok(Some(MapValue::concat(m1, m2))), + } +} + +/// Inserts a key-value pair into the map. If the key already exists, the value is updated. +/// +/// # Example +/// +/// ```slt +/// query T +/// select map_insert(map{'a':1, 'b':2}, 'c', 3); +/// ---- +/// {a:1,b:2,c:3} +/// +/// query T +/// select map_insert(map{'a':1, 'b':2}, 'b', 4); +/// ---- +/// {a:1,b:4} +/// ``` +/// +/// TODO: support variadic arguments +#[function("map_insert(anymap, any, any) -> anymap")] +fn map_insert( + map: MapRef<'_>, + key: Option>, + value: Option>, +) -> MapValue { + let Some(key) = key else { + return map.to_owned(); + }; + MapValue::insert(map, key.into_scalar_impl(), value.to_owned_datum()) +} + +/// Deletes a key-value pair from the map. +/// +/// # Example +/// +/// ```slt +/// query T +/// select map_delete(map{'a':1, 'b':2, 'c':3}, 'b'); +/// ---- +/// {a:1,c:3} +/// +/// query T +/// select map_delete(map{'a':1, 'b':2, 'c':3}, 'd'); +/// ---- +/// {a:1,b:2,c:3} +/// ``` +/// +/// TODO: support variadic arguments +#[function("map_delete(anymap, any) -> anymap")] +fn map_delete(map: MapRef<'_>, key: Option>) -> MapValue { + let Some(key) = key else { + return map.to_owned(); + }; + MapValue::delete(map, key) +} + #[cfg(test)] mod tests { use risingwave_common::array::DataChunk; diff --git a/src/expr/impl/src/scalar/cast.rs b/src/expr/impl/src/scalar/cast.rs index e0dd1a8bb3fc..41c51d95445e 100644 --- a/src/expr/impl/src/scalar/cast.rs +++ b/src/expr/impl/src/scalar/cast.rs @@ -249,7 +249,7 @@ fn map_cast(map: MapRef<'_>, ctx: &Context) -> Result { return_type: ctx.return_type.as_map().clone().into_list(), variadic: ctx.variadic, }; - list_cast(map.into_inner(), &new_ctx).map(MapValue::from_list_entries) + list_cast(map.into_inner(), &new_ctx).map(MapValue::from_entries) } #[cfg(test)] diff --git a/src/frontend/src/binder/expr/function/builtin_scalar.rs b/src/frontend/src/binder/expr/function/builtin_scalar.rs index 824f08cf36b7..73eb722b2601 100644 --- a/src/frontend/src/binder/expr/function/builtin_scalar.rs +++ b/src/frontend/src/binder/expr/function/builtin_scalar.rs @@ -402,6 +402,15 @@ impl Binder { // map ("map_from_entries", raw_call(ExprType::MapFromEntries)), ("map_access",raw_call(ExprType::MapAccess)), + ("map_keys", raw_call(ExprType::MapKeys)), + ("map_values", raw_call(ExprType::MapValues)), + ("map_entries", raw_call(ExprType::MapEntries)), + ("map_from_key_values", raw_call(ExprType::MapFromKeyValues)), + ("map_cat", raw_call(ExprType::MapCat)), + ("map_contains", raw_call(ExprType::MapContains)), + ("map_delete", raw_call(ExprType::MapDelete)), + ("map_insert", raw_call(ExprType::MapInsert)), + ("map_length", raw_call(ExprType::MapLength)), // Functions that return a constant value ("pi", pi()), // greatest and least diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 3c127c7da7c4..85ed93c7dc0c 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -1017,7 +1017,7 @@ pub fn bind_data_type(data_type: &AstDataType) -> Result { AstDataType::Map(kv) => { let key = bind_data_type(&kv.0)?; let value = bind_data_type(&kv.1)?; - DataType::Map(MapType::try_from_kv(key, value)?) + DataType::Map(MapType::try_from_kv(key, value).map_err(ErrorCode::BindError)?) } AstDataType::Custom(qualified_type_name) => { let idents = qualified_type_name diff --git a/src/frontend/src/binder/expr/value.rs b/src/frontend/src/binder/expr/value.rs index 711aa6bbb697..961306408a43 100644 --- a/src/frontend/src/binder/expr/value.rs +++ b/src/frontend/src/binder/expr/value.rs @@ -159,7 +159,7 @@ impl Binder { .into(); let expr: ExprImpl = FunctionCall::new_unchecked( - ExprType::MapFromEntries, + ExprType::MapFromKeyValues, vec![keys, values], DataType::Map(MapType::from_kv(key_type, value_type)), ) @@ -209,7 +209,7 @@ impl Binder { .into(); let expr: ExprImpl = FunctionCall::new_unchecked( - ExprType::MapFromEntries, + ExprType::MapFromKeyValues, vec![keys, values], DataType::Map(map_type), ) diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index 73becd7bc86c..f650fa3cb521 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -17,7 +17,7 @@ use fixedbitset::FixedBitSet; use futures::FutureExt; use paste::paste; use risingwave_common::array::ListValue; -use risingwave_common::types::{DataType, Datum, JsonbVal, Scalar, ScalarImpl}; +use risingwave_common::types::{DataType, Datum, JsonbVal, MapType, Scalar, ScalarImpl}; use risingwave_expr::aggregate::PbAggKind; use risingwave_expr::expr::build_from_prost; use risingwave_pb::expr::expr_node::RexNode; @@ -324,6 +324,19 @@ impl ExprImpl { } } + /// Ensure the return type of this expression is a map of some type. + pub fn try_into_map_type(&self) -> Result { + if self.is_untyped() { + return Err(ErrorCode::BindError( + "could not determine polymorphic type because input has type unknown".into(), + )); + } + match self.return_type() { + DataType::Map(m) => Ok(m), + t => Err(ErrorCode::BindError(format!("expects map but got {t}"))), + } + } + /// Shorthand to enforce implicit cast to boolean pub fn enforce_bool_clause(self, clause: &str) -> RwResult { if self.is_untyped() { diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 59f087672417..3e6c83d8330f 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -251,7 +251,16 @@ impl ExprVisitor for ImpureAnalyzer { | Type::QuoteLiteral | Type::QuoteNullable | Type::MapFromEntries - | Type::MapAccess => + | Type::MapAccess + | Type::MapKeys + | Type::MapValues + | Type::MapEntries + | Type::MapFromKeyValues + | Type::MapCat + | Type::MapContains + | Type::MapDelete + | Type::MapInsert + | Type::MapLength => // expression output is deterministic(same result for the same input) { func_call diff --git a/src/frontend/src/expr/type_inference/func.rs b/src/frontend/src/expr/type_inference/func.rs index 746460e2b636..9ed753049992 100644 --- a/src/frontend/src/expr/type_inference/func.rs +++ b/src/frontend/src/expr/type_inference/func.rs @@ -613,7 +613,7 @@ fn infer_type_for_special( } ExprType::MapAccess => { ensure_arity!("map_access", | inputs | == 2); - let map_type = inputs[0].return_type().into_map(); + let map_type = inputs[0].try_into_map_type()?; // We do not align the map's key type with the input type here, but cast the latter to the former instead. // e.g., for {1:'a'}[1.0], if we align them, we will get "numeric" as the key type, which violates the map type's restriction. match inputs[1].cast_implicit_mut(map_type.key().clone()) { @@ -626,6 +626,40 @@ fn infer_type_for_special( .into()), } } + ExprType::MapCat => { + ensure_arity!("map_contains", | inputs | == 2); + Ok(Some(align_types(inputs.iter_mut())?)) + } + ExprType::MapInsert => { + ensure_arity!("map_insert", | inputs | == 3); + let map_type = inputs[0].try_into_map_type()?; + let rk = inputs[1].cast_implicit_mut(map_type.key().clone()); + let rv = inputs[2].cast_implicit_mut(map_type.value().clone()); + match (rk, rv) { + (Ok(()), Ok(())) => Ok(Some(map_type.into())), + _ => Err(ErrorCode::BindError(format!( + "Cannot insert ({},{}) to {}", + inputs[1].return_type(), + inputs[2].return_type(), + inputs[0].return_type(), + )) + .into()), + } + } + ExprType::MapDelete => { + ensure_arity!("map_delete", | inputs | == 2); + let map_type = inputs[0].try_into_map_type()?; + let rk = inputs[1].cast_implicit_mut(map_type.key().clone()); + match rk { + Ok(()) => Ok(Some(map_type.into())), + _ => Err(ErrorCode::BindError(format!( + "Cannot delete {} from {}", + inputs[1].return_type(), + inputs[0].return_type(), + )) + .into()), + } + } ExprType::Vnode => { ensure_arity!("vnode", 1 <= | inputs |); Ok(Some(VirtualNode::RW_TYPE)) diff --git a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs index 5e9a3ce05392..2c14fc730877 100644 --- a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs +++ b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs @@ -293,6 +293,15 @@ impl Strong { | ExprType::JsonbSet | ExprType::MapFromEntries | ExprType::MapAccess + | ExprType::MapKeys + | ExprType::MapValues + | ExprType::MapEntries + | ExprType::MapFromKeyValues + | ExprType::MapCat + | ExprType::MapContains + | ExprType::MapDelete + | ExprType::MapInsert + | ExprType::MapLength | ExprType::Vnode | ExprType::TestPaidTier | ExprType::Proctime From 11f0acfe548ff0f317c979e51570041058955b9e Mon Sep 17 00:00:00 2001 From: Dylan Date: Mon, 19 Aug 2024 17:42:45 +0800 Subject: [PATCH 22/26] feat(iceberg): bump iceberg-rust to v0.3.0 (#18088) --- Cargo.lock | 209 ++++++++++++++++++++------ Cargo.toml | 6 +- src/connector/src/sink/iceberg/mod.rs | 5 +- 3 files changed, 168 insertions(+), 52 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2ea85047e8b2..0bcef68bbe6a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -230,6 +230,30 @@ dependencies = [ "zstd 0.12.4", ] +[[package]] +name = "apache-avro" +version = "0.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1aef82843a0ec9f8b19567445ad2421ceeb1d711514384bdd3d49fe37102ee13" +dependencies = [ + "bigdecimal 0.4.5", + "digest", + "libflate", + "log", + "num-bigint", + "quad-rand", + "rand", + "regex-lite", + "serde", + "serde_bytes", + "serde_json", + "strum 0.26.3", + "strum_macros 0.26.4", + "thiserror", + "typed-builder 0.19.1", + "uuid", +] + [[package]] name = "apache-avro" version = "0.17.0" @@ -246,7 +270,7 @@ dependencies = [ "regex-lite", "serde", "serde_json", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "thiserror", "typed-builder 0.18.2", @@ -5926,11 +5950,11 @@ dependencies = [ [[package]] name = "iceberg" -version = "0.2.0" -source = "git+https://github.com/risingwavelabs/iceberg-rust.git?rev=24bd5869f2779a8b9786b5a6e1f9723844f5a82c#24bd5869f2779a8b9786b5a6e1f9723844f5a82c" +version = "0.3.0" +source = "git+https://github.com/apache/iceberg-rust.git?rev=4440af69a354d9af56f239a6126a7f4b7945d58b#4440af69a354d9af56f239a6126a7f4b7945d58b" dependencies = [ "anyhow", - "apache-avro 0.17.0", + "apache-avro 0.17.0 (registry+https://github.com/rust-lang/crates.io-index)", "array-init", "arrow-arith 52.0.0", "arrow-array 52.0.0", @@ -5938,22 +5962,18 @@ dependencies = [ "arrow-schema 52.0.0", "arrow-select 52.0.0", "arrow-string 52.0.0", - "async-stream", "async-trait", "bimap", "bitvec", "bytes", "chrono", "derive_builder 0.20.0", - "either", "fnv", "futures", "itertools 0.13.0", - "lazy_static", - "log", "murmur3", "once_cell", - "opendal", + "opendal 0.49.0", "ordered-float 4.1.1", "parquet 52.0.0", "reqwest 0.12.4", @@ -5965,19 +5985,19 @@ dependencies = [ "serde_repr", "serde_with 3.8.0", "tokio", - "typed-builder 0.18.2", + "typed-builder 0.19.1", "url", - "urlencoding", "uuid", ] [[package]] name = "iceberg-catalog-rest" -version = "0.2.0" -source = "git+https://github.com/risingwavelabs/iceberg-rust.git?rev=24bd5869f2779a8b9786b5a6e1f9723844f5a82c#24bd5869f2779a8b9786b5a6e1f9723844f5a82c" +version = "0.3.0" +source = "git+https://github.com/apache/iceberg-rust.git?rev=4440af69a354d9af56f239a6126a7f4b7945d58b#4440af69a354d9af56f239a6126a7f4b7945d58b" dependencies = [ "async-trait", "chrono", + "http 1.1.0", "iceberg", "itertools 0.13.0", "log", @@ -5985,8 +6005,8 @@ dependencies = [ "serde", "serde_derive", "serde_json", - "typed-builder 0.18.2", - "urlencoding", + "tokio", + "typed-builder 0.19.1", "uuid", ] @@ -5996,7 +6016,7 @@ version = "0.3.141592654" source = "git+https://github.com/risingwavelabs/icelake.git?rev=1860eb315183a5f3f72b4097c1e40d49407f8373#1860eb315183a5f3f72b4097c1e40d49407f8373" dependencies = [ "anyhow", - "apache-avro 0.17.0", + "apache-avro 0.17.0 (git+https://github.com/apache/avro.git)", "arrow-arith 52.0.0", "arrow-array 52.0.0", "arrow-buffer 52.0.0", @@ -6019,7 +6039,7 @@ dependencies = [ "log", "murmur3", "once_cell", - "opendal", + "opendal 0.47.2", "ordered-float 3.9.1", "parquet 52.0.0", "prometheus", @@ -6696,9 +6716,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.21" +version = "0.4.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" +checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24" dependencies = [ "value-bag", ] @@ -7542,9 +7562,9 @@ dependencies = [ [[package]] name = "num-bigint" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c165a9ab64cf766f73521c0dd2cfdff64f488b8f0b3e621face3462d3db536d7" +checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" dependencies = [ "num-integer", "num-traits", @@ -7793,7 +7813,7 @@ dependencies = [ "percent-encoding", "prometheus", "quick-xml 0.31.0", - "reqsign", + "reqsign 0.15.2", "reqwest 0.12.4", "serde", "serde_json", @@ -7802,6 +7822,36 @@ dependencies = [ "uuid", ] +[[package]] +name = "opendal" +version = "0.49.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39d516adf7db912c38af382c3e92c27cd62fbbc240e630920555d784c2ab1494" +dependencies = [ + "anyhow", + "async-trait", + "backon", + "base64 0.22.0", + "bytes", + "chrono", + "crc32c", + "flagset", + "futures", + "getrandom", + "http 1.1.0", + "log", + "md-5", + "once_cell", + "percent-encoding", + "quick-xml 0.36.1", + "reqsign 0.16.0", + "reqwest 0.12.4", + "serde", + "serde_json", + "tokio", + "uuid", +] + [[package]] name = "openidconnect" version = "3.4.0" @@ -9337,6 +9387,26 @@ dependencies = [ "serde", ] +[[package]] +name = "quick-xml" +version = "0.35.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "86e446ed58cef1bbfe847bc2fda0e2e4ea9f0e57b90c507d4781292590d72a4e" +dependencies = [ + "memchr", + "serde", +] + +[[package]] +name = "quick-xml" +version = "0.36.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96a05e2e8efddfa51a84ca47cec303fac86c8541b686d37cac5efc0e094417bc" +dependencies = [ + "memchr", + "serde", +] + [[package]] name = "quote" version = "1.0.36" @@ -9631,6 +9701,34 @@ dependencies = [ "sha2", ] +[[package]] +name = "reqsign" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "03dd4ba7c3901dd43e6b8c7446a760d45bc1ea4301002e1a6fa48f97c3a796fa" +dependencies = [ + "anyhow", + "async-trait", + "base64 0.22.0", + "chrono", + "form_urlencoded", + "getrandom", + "hex", + "hmac", + "home", + "http 1.1.0", + "log", + "percent-encoding", + "quick-xml 0.35.0", + "rand", + "reqwest 0.12.4", + "rust-ini", + "serde", + "serde_json", + "sha1", + "sha2", +] + [[package]] name = "reqwest" version = "0.11.20" @@ -9918,7 +10016,7 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "memcomparable", - "opendal", + "opendal 0.47.2", "parking_lot 0.12.1", "parquet 52.0.0", "paste", @@ -10035,7 +10133,7 @@ dependencies = [ "risingwave_meta_node", "risingwave_rt", "shell-words", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "tempfile", "thiserror-ext", @@ -10144,7 +10242,7 @@ dependencies = [ "speedate", "stacker", "static_assertions", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "sysinfo", "tempfile", @@ -10452,7 +10550,7 @@ dependencies = [ "mysql_common", "nexmark", "num-bigint", - "opendal", + "opendal 0.47.2", "openssl", "parking_lot 0.12.1", "parquet 52.0.0", @@ -10494,7 +10592,7 @@ dependencies = [ "serde_yaml", "simd-json", "sqlx", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "syn 1.0.109", "tempfile", @@ -11065,7 +11163,7 @@ dependencies = [ "sea-orm", "serde", "serde_json", - "strum 0.26.2", + "strum 0.26.3", "sync-point", "thiserror", "thiserror-ext", @@ -11212,7 +11310,7 @@ dependencies = [ "madsim", "madsim-aws-sdk-s3", "madsim-tokio", - "opendal", + "opendal 0.47.2", "prometheus", "reqwest 0.12.4", "risingwave_common", @@ -11240,7 +11338,7 @@ dependencies = [ "prost-helpers", "risingwave_error", "serde", - "strum 0.26.2", + "strum 0.26.3", "thiserror", "walkdir", "workspace-hack", @@ -11598,7 +11696,7 @@ dependencies = [ "serde_yaml", "smallvec", "static_assertions", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "thiserror", "thiserror-ext", @@ -12478,9 +12576,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.203" +version = "1.0.208" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" +checksum = "cff085d2cb684faa248efb494c39b68e522822ac0de72ccf08109abde717cfb2" dependencies = [ "serde_derive", ] @@ -12506,9 +12604,9 @@ dependencies = [ [[package]] name = "serde_bytes" -version = "0.11.12" +version = "0.11.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab33ec92f677585af6d88c65593ae2375adde54efdbf16d597f2cbc7a6d368ff" +checksum = "387cc504cb06bb40a96c8e04e951fe01854cf6bc921053c954e4a606d9675c6a" dependencies = [ "serde", ] @@ -12527,9 +12625,9 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.203" +version = "1.0.208" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" +checksum = "24008e81ff7613ed8e5ba0cfaf24e2c2f1e5b8a0495711e44fcd4882fca62bcf" dependencies = [ "proc-macro2", "quote", @@ -12549,11 +12647,12 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.117" +version = "1.0.125" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "455182ea6142b14f93f4bc5320a2b31c1f266b66a4a5c858b013302a5d8cbfc3" +checksum = "83c8e735a073ccf5be70aa8066aa984eaf2fa000db6c8d0100ae605b366d31ed" dependencies = [ "itoa", + "memchr", "ryu", "serde", ] @@ -13445,9 +13544,9 @@ dependencies = [ [[package]] name = "strum" -version = "0.26.2" +version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d8cec3501a5194c432b2b7976db6b7d10ec95c253208b45f83f7136aa985e29" +checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" dependencies = [ "strum_macros 0.26.4", ] @@ -13692,9 +13791,9 @@ checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76" [[package]] name = "thiserror" -version = "1.0.61" +version = "1.0.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c546c80d6be4bc6a00c0f01730c08df82eaa7a7a61f11d656526506112cc1709" +checksum = "c0342370b38b6a11b6cc11d6a805569958d54cfa061a29969c3b5ce2ea405724" dependencies = [ "thiserror-impl", ] @@ -13723,9 +13822,9 @@ dependencies = [ [[package]] name = "thiserror-impl" -version = "1.0.61" +version = "1.0.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" +checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" dependencies = [ "proc-macro2", "quote", @@ -14617,6 +14716,15 @@ dependencies = [ "typed-builder-macro 0.18.2", ] +[[package]] +name = "typed-builder" +version = "0.19.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a06fbd5b8de54c5f7c91f6fe4cebb949be2125d7758e630bb58b1d831dbce600" +dependencies = [ + "typed-builder-macro 0.19.1", +] + [[package]] name = "typed-builder-macro" version = "0.16.2" @@ -14639,6 +14747,17 @@ dependencies = [ "syn 2.0.66", ] +[[package]] +name = "typed-builder-macro" +version = "0.19.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9534daa9fd3ed0bd911d462a37f172228077e7abf18c18a5f67199d959205f8" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.66", +] + [[package]] name = "typenum" version = "1.16.0" diff --git a/Cargo.toml b/Cargo.toml index 22ee9c6a1d01..35927b31aa78 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -141,10 +141,8 @@ arrow-array-iceberg = { package = "arrow-array", version = "52" } arrow-schema-iceberg = { package = "arrow-schema", version = "52" } arrow-buffer-iceberg = { package = "arrow-buffer", version = "52" } arrow-cast-iceberg = { package = "arrow-cast", version = "52" } -# TODO -# After apache/iceberg-rust#411 is merged, we move to the upstream version. -iceberg = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "24bd5869f2779a8b9786b5a6e1f9723844f5a82c" } -iceberg-catalog-rest = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "24bd5869f2779a8b9786b5a6e1f9723844f5a82c" } +iceberg = { git = "https://github.com/apache/iceberg-rust.git", rev = "4440af69a354d9af56f239a6126a7f4b7945d58b" } +iceberg-catalog-rest = { git = "https://github.com/apache/iceberg-rust.git", rev = "4440af69a354d9af56f239a6126a7f4b7945d58b" } opendal = "0.47" arrow-array = "50" arrow-arith = "50" diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index a17c98985de0..540fea13b6c0 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -501,7 +501,7 @@ impl IcebergConfig { .map_err(|e| SinkError::Iceberg(anyhow!(e))) } - async fn create_catalog_v2(&self) -> ConnectorResult> { + fn create_catalog_v2(&self) -> ConnectorResult> { match self.catalog_type() { "storage" => { let config = StorageCatalogConfig::builder() @@ -536,7 +536,7 @@ impl IcebergConfig { })?) .props(iceberg_configs) .build(); - let catalog = iceberg_catalog_rest::RestCatalog::new(config).await?; + let catalog = iceberg_catalog_rest::RestCatalog::new(config); Ok(Arc::new(catalog)) } catalog_type @@ -570,7 +570,6 @@ impl IcebergConfig { pub async fn load_table_v2(&self) -> ConnectorResult { let catalog = self .create_catalog_v2() - .await .context("Unable to load iceberg catalog")?; let table_id = self From d5e832cbe0339f5deedab3baa18b8f0df8f91772 Mon Sep 17 00:00:00 2001 From: Kevin Axel Date: Mon, 19 Aug 2024 20:38:29 +0800 Subject: [PATCH 23/26] feat(test): switch meta to sql backend in simluate test (#15612) Signed-off-by: Kevin Axel Co-authored-by: Noel Kwan Co-authored-by: Noel Kwan <47273164+kwannoel@users.noreply.github.com> --- .gitignore | 3 + Cargo.lock | 108 ++++++++++++---------- Cargo.toml | 6 +- ci/scripts/deterministic-recovery-test.sh | 28 ++++-- ci/workflows/main-cron.yml | 26 +++++- src/object_store/Cargo.toml | 2 +- src/tests/simulation/Cargo.toml | 3 +- src/tests/simulation/src/cluster.rs | 85 ++++++++++++----- src/tests/simulation/src/main.rs | 7 +- 9 files changed, 182 insertions(+), 86 deletions(-) diff --git a/.gitignore b/.gitignore index 25397584e2ef..72e3ed1487fe 100644 --- a/.gitignore +++ b/.gitignore @@ -80,6 +80,9 @@ e2e_test/iceberg/spark-*-bin* **/poetry.lock +**/*.sqlite +**/*.sqlite-journal + *.slt.temp .direnv/ diff --git a/Cargo.lock b/Cargo.lock index 0bcef68bbe6a..d007b6c6f4f3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1227,6 +1227,16 @@ version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1181e1e0d1fce796a03db1ae795d67167da795f9cf4a39c37589e85ef57f26d3" +[[package]] +name = "atomic-write-file" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cbf54d4588732bdfc5ebc3eb9f74f20e027112fc31de412fc7ff0cd1c6896dae" +dependencies = [ + "nix 0.28.0", + "rand", +] + [[package]] name = "auto_enums" version = "0.8.5" @@ -2159,7 +2169,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a6362ed55def622cddc70a4746a68554d7b687713770de539e59a739b249f8ed" dependencies = [ "borsh-derive", - "cfg_aliases", + "cfg_aliases 0.2.1", ] [[package]] @@ -2516,6 +2526,12 @@ dependencies = [ "syn 2.0.66", ] +[[package]] +name = "cfg_aliases" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fd16c4719339c4530435d38e511904438d07cce7950afa3718a84ac36c10e89e" + [[package]] name = "cfg_aliases" version = "0.2.1" @@ -4741,18 +4757,6 @@ dependencies = [ "num-traits", ] -[[package]] -name = "flume" -version = "0.10.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1657b4441c3403d9f7b3409e47575237dac27b1b5726df654a6ecbf92f0f7577" -dependencies = [ - "futures-core", - "futures-sink", - "pin-project", - "spin 0.9.8", -] - [[package]] name = "flume" version = "0.11.0" @@ -6611,9 +6615,9 @@ dependencies = [ [[package]] name = "libsqlite3-sys" -version = "0.26.0" +version = "0.27.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "afc22eff61b133b115c6e8c74e818c628d6d5e7a502afea6f64dee076dd94326" +checksum = "cf4e226dcd58b4be396f7bd3c20da8fdee2911400705297ba7d2d7cc2c30f716" dependencies = [ "cc", "pkg-config", @@ -6905,9 +6909,9 @@ dependencies = [ [[package]] name = "madsim-tokio" -version = "0.2.28" +version = "0.2.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "569929c869275edc1e2c1f1381a688a6e5a4200302b58caff819e07414ccddb9" +checksum = "53a36782df7996da4863e04526454fbd43067fac1a4dcb8fc663acd538cbf6af" dependencies = [ "madsim", "spin 0.9.8", @@ -7453,6 +7457,18 @@ dependencies = [ "libc", ] +[[package]] +name = "nix" +version = "0.28.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ab2156c4fce2f8df6c499cc1c763e4394b7482525bf2a9701c9d79d215f519e4" +dependencies = [ + "bitflags 2.6.0", + "cfg-if", + "cfg_aliases 0.1.1", + "libc", +] + [[package]] name = "nix" version = "0.29.0" @@ -7461,7 +7477,7 @@ checksum = "71e2746dc3a24dd78b3cfcb7be93368c6de9963d30f43a6a73998a9cf4b17b46" dependencies = [ "bitflags 2.6.0", "cfg-if", - "cfg_aliases", + "cfg_aliases 0.2.1", "libc", ] @@ -9040,7 +9056,7 @@ checksum = "8bdf592881d821b83d471f8af290226c8d51402259e9bb5be7f9f8bdebbb11ac" dependencies = [ "bytes", "heck 0.4.1", - "itertools 0.11.0", + "itertools 0.10.5", "log", "multimap 0.8.3", "once_cell", @@ -9095,7 +9111,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "265baba7fabd416cf5078179f7d2cbeca4ce7a9041111900675ea7c4cb8a4c32" dependencies = [ "anyhow", - "itertools 0.11.0", + "itertools 0.10.5", "proc-macro2", "quote", "syn 2.0.66", @@ -9278,7 +9294,7 @@ dependencies = [ "indoc", "libc", "memoffset", - "parking_lot 0.12.1", + "parking_lot 0.11.2", "portable-atomic", "pyo3-build-config", "pyo3-ffi", @@ -11496,6 +11512,7 @@ dependencies = [ "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "tracing", "tracing-subscriber", + "uuid", ] [[package]] @@ -11843,7 +11860,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e1568e15fab2d546f940ed3a21f48bbbd1c494c90c99c4481339364a497f94a9" dependencies = [ "bytes", - "flume 0.11.0", + "flume", "futures-util", "log", "rustls-native-certs 0.7.0", @@ -13243,9 +13260,8 @@ dependencies = [ [[package]] name = "sqlx" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e58421b6bc416714d5115a2ca953718f6c621a51b68e4f4922aea5a4391a721" +version = "0.7.3" +source = "git+https://github.com/kwannoel/sqlx.git?rev=5a32ad8ce842176c202df30910e1416655eda071#5a32ad8ce842176c202df30910e1416655eda071" dependencies = [ "sqlx-core", "sqlx-macros", @@ -13256,9 +13272,8 @@ dependencies = [ [[package]] name = "sqlx-core" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd4cef4251aabbae751a3710927945901ee1d97ee96d757f6880ebb9a79bfd53" +version = "0.7.3" +source = "git+https://github.com/kwannoel/sqlx.git?rev=5a32ad8ce842176c202df30910e1416655eda071#5a32ad8ce842176c202df30910e1416655eda071" dependencies = [ "ahash 0.8.11", "atoi", @@ -13268,7 +13283,6 @@ dependencies = [ "chrono", "crc", "crossbeam-queue", - "dotenvy", "either", "event-listener 2.5.3", "futures-channel", @@ -13280,6 +13294,7 @@ dependencies = [ "hex", "indexmap 2.2.6", "log", + "madsim-tokio", "memchr", "native-tls", "once_cell", @@ -13293,7 +13308,6 @@ dependencies = [ "sqlformat", "thiserror", "time", - "tokio", "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", "tracing", "url", @@ -13302,9 +13316,8 @@ dependencies = [ [[package]] name = "sqlx-macros" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "208e3165167afd7f3881b16c1ef3f2af69fa75980897aac8874a0696516d12c2" +version = "0.7.3" +source = "git+https://github.com/kwannoel/sqlx.git?rev=5a32ad8ce842176c202df30910e1416655eda071#5a32ad8ce842176c202df30910e1416655eda071" dependencies = [ "proc-macro2", "quote", @@ -13315,14 +13328,15 @@ dependencies = [ [[package]] name = "sqlx-macros-core" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a4a8336d278c62231d87f24e8a7a74898156e34c1c18942857be2acb29c7dfc" +version = "0.7.3" +source = "git+https://github.com/kwannoel/sqlx.git?rev=5a32ad8ce842176c202df30910e1416655eda071#5a32ad8ce842176c202df30910e1416655eda071" dependencies = [ + "atomic-write-file", "dotenvy", "either", "heck 0.4.1", "hex", + "madsim-tokio", "once_cell", "proc-macro2", "quote", @@ -13335,15 +13349,13 @@ dependencies = [ "sqlx-sqlite", "syn 1.0.109", "tempfile", - "tokio", "url", ] [[package]] name = "sqlx-mysql" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ca69bf415b93b60b80dc8fda3cb4ef52b2336614d8da2de5456cc942a110482" +version = "0.7.3" +source = "git+https://github.com/kwannoel/sqlx.git?rev=5a32ad8ce842176c202df30910e1416655eda071#5a32ad8ce842176c202df30910e1416655eda071" dependencies = [ "atoi", "base64 0.21.7", @@ -13388,9 +13400,8 @@ dependencies = [ [[package]] name = "sqlx-postgres" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0db2df1b8731c3651e204629dd55e52adbae0462fa1bdcbed56a2302c18181e" +version = "0.7.3" +source = "git+https://github.com/kwannoel/sqlx.git?rev=5a32ad8ce842176c202df30910e1416655eda071#5a32ad8ce842176c202df30910e1416655eda071" dependencies = [ "atoi", "base64 0.21.7", @@ -13433,26 +13444,27 @@ dependencies = [ [[package]] name = "sqlx-sqlite" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be4c21bf34c7cae5b283efb3ac1bcc7670df7561124dc2f8bdc0b59be40f79a2" +version = "0.7.3" +source = "git+https://github.com/kwannoel/sqlx.git?rev=5a32ad8ce842176c202df30910e1416655eda071#5a32ad8ce842176c202df30910e1416655eda071" dependencies = [ + "async-channel 2.2.1", "atoi", "chrono", - "flume 0.10.14", + "flume", "futures-channel", "futures-core", - "futures-executor", "futures-intrusive", "futures-util", "libsqlite3-sys", "log", + "madsim-tokio", "percent-encoding", "serde", "sqlx-core", "time", "tracing", "url", + "urlencoding", "uuid", ] diff --git a/Cargo.toml b/Cargo.toml index 35927b31aa78..8c468bf059d5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -193,7 +193,7 @@ sea-orm = { version = "0.12.14", features = [ "sqlx-sqlite", "runtime-tokio-native-tls", ] } -sqlx = "0.7" +sqlx = { version = "0.7.3", default-features = false, features = ["bigdecimal", "chrono", "json", "mysql", "postgres", "runtime-tokio-native-tls", "rust_decimal", "sqlite", "time", "uuid"] } tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "0dd1055", features = ["net", "fs"] } tokio-util = "0.7" tracing-opentelemetry = "0.24" @@ -340,6 +340,10 @@ getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "e79a7ae # tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "0dd1055" } tokio-retry = { git = "https://github.com/madsim-rs/rust-tokio-retry.git", rev = "95e2fd3" } tokio-postgres = { git = "https://github.com/madsim-rs/rust-postgres.git", rev = "ac00d88" } +# NOTE(kwannoel): patch sqlx to use a version compatible with madsim. +# If we can merge in: https://github.com/madsim-rs/sqlx/pull/2, +# we can change it to patch madsim version instead. +sqlx = { git = "https://github.com/kwannoel/sqlx.git", rev = "5a32ad8ce842176c202df30910e1416655eda071" } futures-timer = { git = "https://github.com/madsim-rs/futures-timer.git", rev = "05b33b4" } # patch to remove preserve_order from serde_json bson = { git = "https://github.com/risingwavelabs/bson-rust", rev = "e5175ec" } diff --git a/ci/scripts/deterministic-recovery-test.sh b/ci/scripts/deterministic-recovery-test.sh index 2afe2f03b956..ced644f4e487 100755 --- a/ci/scripts/deterministic-recovery-test.sh +++ b/ci/scripts/deterministic-recovery-test.sh @@ -19,7 +19,8 @@ risingwave_meta::barrier::progress=debug" # Extra logs you can enable if the existing trace does not give enough info. #risingwave_stream::executor::backfill=trace, -#risingwave_meta::barrier::progress=debug, +#risingwave_meta::barrier::progress=debug +#risingwave_meta::controller::streaming_job=trace # ========= Some tips for debugging recovery tests ========= # 1. If materialized view failed to create after multiple retries @@ -44,11 +45,26 @@ filter_stack_trace_for_all_logs() { trap filter_stack_trace_for_all_logs ERR +# NOTE(kwannoel): We must use `export` here, because the variables are not substituted +# directly via bash subtitution. Instead, the `parallel` command substitutes the variables +# from the environment. If they are declared without `export`, `parallel` can't read them from the env. +export EXTRA_ARGS="" + +if [[ -n "${USE_SQL_BACKEND:-}" ]]; then + export EXTRA_ARGS="--sqlite-data-dir=." +fi + +if [[ -n "${USE_ARRANGEMENT_BACKFILL:-}" ]]; then + export EXTRA_ARGS="$EXTRA_ARGS --use-arrangement-backfill" +fi + +echo "--- EXTRA_ARGS: ${EXTRA_ARGS}" + echo "--- deterministic simulation e2e, ci-3cn-2fe-3meta, recovery, background_ddl" seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ --kill \ --kill-rate=${KILL_RATE} \ -${USE_ARRANGEMENT_BACKFILL:-} \ +${EXTRA_ARGS:-} \ ./e2e_test/background_ddl/sim/basic.slt \ 2> $LOGDIR/recovery-background-ddl-{}.log && rm $LOGDIR/recovery-background-ddl-{}.log' @@ -57,7 +73,7 @@ seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ --kill \ --kill-rate=${KILL_RATE} \ --background-ddl-rate=${BACKGROUND_DDL_RATE} \ -${USE_ARRANGEMENT_BACKFILL:-} \ +${EXTRA_ARGS:-} \ ./e2e_test/ddl/\*\*/\*.slt 2> $LOGDIR/recovery-ddl-{}.log && rm $LOGDIR/recovery-ddl-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe-3meta, recovery, streaming" @@ -65,7 +81,7 @@ seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ --kill \ --kill-rate=${KILL_RATE} \ --background-ddl-rate=${BACKGROUND_DDL_RATE} \ -${USE_ARRANGEMENT_BACKFILL:-} \ +${EXTRA_ARGS:-} \ ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/recovery-streaming-{}.log && rm $LOGDIR/recovery-streaming-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe-3meta, recovery, batch" @@ -73,7 +89,7 @@ seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ --kill \ --kill-rate=${KILL_RATE} \ --background-ddl-rate=${BACKGROUND_DDL_RATE} \ -${USE_ARRANGEMENT_BACKFILL:-} \ +${EXTRA_ARGS:-} \ ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/recovery-batch-{}.log && rm $LOGDIR/recovery-batch-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe-3meta, recovery, kafka source,sink" @@ -81,5 +97,5 @@ seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ --kill \ --kill-rate=${KILL_RATE} \ --kafka-datadir=./scripts/source/test_data \ -${USE_ARRANGEMENT_BACKFILL:-} \ +${EXTRA_ARGS:-} \ ./e2e_test/source/basic/kafka\*.slt 2> $LOGDIR/recovery-source-{}.log && rm $LOGDIR/recovery-source-{}.log' \ No newline at end of file diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 34155618bf9a..ac5b27eee773 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -367,7 +367,27 @@ steps: timeout_in_minutes: 80 retry: *auto-retry - - label: "recovery test (madsim)" + # sql backend recovery tests + - label: "recovery test (sql,madsim)" + key: "recovery-test-deterministic-sql" + command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.0 USE_SQL_BACKEND=true timeout 65m ci/scripts/deterministic-recovery-test.sh" + # NOTE(kwannoel): It will only run when the recovery tests label is added currently. + # This is because there are currently some bugs which cause the test to fail. + if: | + build.pull_request.labels includes "ci/run-sql-recovery-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)sql-recovery-tests?-deterministic-simulation(,|$$)/ + depends_on: "build-simulation" + plugins: + - docker-compose#v5.1.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + # Only upload zipped files, otherwise the logs is too much. + - ./ci/plugins/upload-failure-logs-zipped + timeout_in_minutes: 70 + retry: *auto-retry + + - label: "recovery test (etcd,madsim)" key: "recovery-test-deterministic" command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.0 timeout 65m ci/scripts/deterministic-recovery-test.sh" if: | @@ -386,9 +406,9 @@ steps: retry: *auto-retry # Ddl statements will randomly run with background_ddl. - - label: "background_ddl, arrangement_backfill recovery test (madsim)" + - label: "background_ddl, arrangement_backfill recovery test (etcd,madsim)" key: "background-ddl-arrangement-backfill-recovery-test-deterministic" - command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.8 USE_ARRANGEMENT_BACKFILL=--use-arrangement-backfill timeout 65m ci/scripts/deterministic-recovery-test.sh" + command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.8 USE_ARRANGEMENT_BACKFILL=true timeout 65m ci/scripts/deterministic-recovery-test.sh" if: | !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation" diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index fa433a30abcb..912eb4073130 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -30,7 +30,7 @@ hyper = { version = "0.14", features = ["tcp", "client"] } # hyper-rustls = { version = "0.24.2", features = ["webpki-roots"] } hyper-tls = "0.5.0" itertools = { workspace = true } -madsim = "0.2.27" +madsim = "0.2.30" opendal = { workspace = true, features = [ "executors-tokio", "services-azblob", diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index 143d0f0c01c7..8729207c0d02 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -24,7 +24,7 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] } glob = "0.3" itertools = { workspace = true } lru = { workspace = true } -madsim = "0.2.27" +madsim = "0.2.30" paste = "1" pin-project = "1.1" pretty_assertions = "1" @@ -58,6 +58,7 @@ tokio-postgres = "0.7" tokio-stream = { workspace = true } tracing = "0.1" tracing-subscriber = { version = "0.3", features = ["env-filter"] } +uuid = "*" [lints] workspace = true diff --git a/src/tests/simulation/src/cluster.rs b/src/tests/simulation/src/cluster.rs index 6c9db8c48170..26fdc3a8757e 100644 --- a/src/tests/simulation/src/cluster.rs +++ b/src/tests/simulation/src/cluster.rs @@ -39,6 +39,7 @@ use risingwave_pb::common::WorkerNode; use sqllogictest::AsyncDB; #[cfg(not(madsim))] use tokio::runtime::Handle; +use uuid::Uuid; use crate::client::RisingWave; @@ -93,6 +94,9 @@ pub struct Configuration { /// Queries to run per session. pub per_session_queries: Arc>, + + /// dir to store SQL backend sqlite db + pub sqlite_data_dir: Option, } impl Default for Configuration { @@ -122,6 +126,7 @@ metrics_level = "Disabled" etcd_timeout_rate: 0.0, etcd_data_path: None, per_session_queries: vec![].into(), + sqlite_data_dir: None, } } } @@ -356,10 +361,16 @@ impl Cluster { println!("seed = {}", handle.seed()); println!("{:#?}", conf); + if conf.sqlite_data_dir.is_some() && conf.etcd_data_path.is_some() { + bail!("sqlite_data_dir and etcd_data_path cannot be set at the same time"); + } + // setup DNS and load balance let net = madsim::net::NetSim::current(); - net.add_dns_record("etcd", "192.168.10.1".parse().unwrap()); for i in 1..=conf.meta_nodes { + if conf.sqlite_data_dir.is_none() { + net.add_dns_record("etcd", "192.168.10.1".parse().unwrap()); + } net.add_dns_record( &format!("meta-{i}"), format!("192.168.1.{i}").parse().unwrap(), @@ -380,24 +391,26 @@ impl Cluster { } // etcd node - let etcd_data = conf - .etcd_data_path - .as_ref() - .map(|path| std::fs::read_to_string(path).unwrap()); - handle - .create_node() - .name("etcd") - .ip("192.168.10.1".parse().unwrap()) - .init(move || { - let addr = "0.0.0.0:2388".parse().unwrap(); - let mut builder = - etcd_client::SimServer::builder().timeout_rate(conf.etcd_timeout_rate); - if let Some(data) = &etcd_data { - builder = builder.load(data.clone()); - } - builder.serve(addr) - }) - .build(); + if conf.sqlite_data_dir.is_none() { + let etcd_data = conf + .etcd_data_path + .as_ref() + .map(|path| std::fs::read_to_string(path).unwrap()); + handle + .create_node() + .name("etcd") + .ip("192.168.10.1".parse().unwrap()) + .init(move || { + let addr = "0.0.0.0:2388".parse().unwrap(); + let mut builder = + etcd_client::SimServer::builder().timeout_rate(conf.etcd_timeout_rate); + if let Some(data) = &etcd_data { + builder = builder.load(data.clone()); + } + builder.serve(addr) + }) + .build(); + } // kafka broker handle @@ -432,9 +445,33 @@ impl Cluster { } std::env::set_var("RW_META_ADDR", meta_addrs.join(",")); + let mut sql_endpoint = String::new(); + let mut backend_args = if let Some(sqlite_data_dir) = conf.sqlite_data_dir.as_ref() { + sql_endpoint = format!( + "sqlite://{}stest-{}.sqlite?mode=rwc", + sqlite_data_dir.display(), + Uuid::new_v4() + ); + vec!["--backend", "sql", "--sql-endpoint", &sql_endpoint] + } else { + vec!["--backend", "etcd", "--etcd-endpoints", "etcd:2388"] + }; + + // FIXME(kwannoel): + // Currently we just use the on-disk version, + // but it can lead to randomness due to disk io. + // We can use shared in-memory db instead. + // However sqlite cannot be started inside meta. + // Because if cluster stops, then this db will be dropped. + // We must instantiate it outside, not just pass the path in. + // let sqlite_path = format!( + // "sqlite::file:memdb{}?mode=memory&cache=shared", + // Uuid::new_v4() + // ); + // meta node for i in 1..=conf.meta_nodes { - let opts = risingwave_meta_node::MetaNodeOpts::parse_from([ + let args = [ "meta-node", "--config-path", conf.config_path.as_str(), @@ -442,17 +479,15 @@ impl Cluster { "0.0.0.0:5690", "--advertise-addr", &format!("meta-{i}:5690"), - "--backend", - "etcd", - "--etcd-endpoints", - "etcd:2388", "--state-store", "hummock+sim://hummockadmin:hummockadmin@192.168.12.1:9301/hummock001", "--data-directory", "hummock_001", "--temp-secret-file-dir", &format!("./secrets/meta-{i}"), - ]); + ]; + let args = args.into_iter().chain(backend_args.clone().into_iter()); + let opts = risingwave_meta_node::MetaNodeOpts::parse_from(args); handle .create_node() .name(format!("meta-{i}")) diff --git a/src/tests/simulation/src/main.rs b/src/tests/simulation/src/main.rs index e45c4a3285ac..102db8ccc1c4 100644 --- a/src/tests/simulation/src/main.rs +++ b/src/tests/simulation/src/main.rs @@ -136,6 +136,10 @@ pub struct Args { #[clap(long)] etcd_dump: Option, + /// dir to store sqlite backend data of meta node + #[clap(long)] + sqlite_data_dir: Option, + #[arg(short, long)] e2e_extended_test: bool, @@ -144,7 +148,7 @@ pub struct Args { #[clap(long, default_value = "0.0")] background_ddl_rate: f64, - /// Use arrangement backfill by default + /// Use arrangement backfill #[clap(long, default_value = "false")] use_arrangement_backfill: bool, } @@ -176,6 +180,7 @@ async fn main() { meta_nodes: args.meta_nodes, etcd_timeout_rate: args.etcd_timeout_rate, etcd_data_path: args.etcd_data, + sqlite_data_dir: args.sqlite_data_dir, per_session_queries: if args.use_arrangement_backfill { vec!["SET STREAMING_USE_ARRANGEMENT_BACKFILL = true;".to_string()].into() } else { From 0595aa1d901b2f99811ba1e7e40161b395ddd6a4 Mon Sep 17 00:00:00 2001 From: ka-weihe Date: Mon, 19 Aug 2024 18:44:08 +0200 Subject: [PATCH 24/26] feat(connector): add support for path-style access in Iceberg sink connector (#17747) Co-authored-by: ka-weihe Co-authored-by: lmatz --- src/connector/src/sink/iceberg/mod.rs | 22 +++++++++++++++++++++- src/connector/with_options_sink.yaml | 4 ++++ 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index 540fea13b6c0..2274b5805d81 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -123,6 +123,13 @@ pub struct IcebergConfig { #[serde(rename = "s3.secret.key")] pub secret_key: String, + #[serde( + rename = "s3.path.style.access", + default, + deserialize_with = "deserialize_bool_from_string" + )] + pub path_style_access: bool, + #[serde( rename = "primary_key", default, @@ -270,6 +277,10 @@ impl IcebergConfig { "iceberg.table.io.secret_access_key".to_string(), self.secret_key.clone().to_string(), ); + iceberg_configs.insert( + "iceberg.table.io.enable_virtual_host_style".to_string(), + (!self.path_style_access).to_string(), + ); let (bucket, root) = { let url = Url::parse(&self.path).map_err(|e| SinkError::Iceberg(anyhow!(e)))?; @@ -409,7 +420,10 @@ impl IcebergConfig { "s3.secret-access-key".to_string(), self.secret_key.clone().to_string(), ); - + java_catalog_configs.insert( + "s3.path-style-access".to_string(), + self.path_style_access.to_string(), + ); if matches!(self.catalog_type.as_deref(), Some("glue")) { java_catalog_configs.insert( "client.credentials-provider".to_string(), @@ -1286,6 +1300,7 @@ mod test { ("s3.endpoint", "http://127.0.0.1:9301"), ("s3.access.key", "hummockadmin"), ("s3.secret.key", "hummockadmin"), + ("s3.path.style.access", "true"), ("s3.region", "us-east-1"), ("catalog.type", "jdbc"), ("catalog.name", "demo"), @@ -1315,6 +1330,7 @@ mod test { endpoint: Some("http://127.0.0.1:9301".to_string()), access_key: "hummockadmin".to_string(), secret_key: "hummockadmin".to_string(), + path_style_access: true, primary_key: Some(vec!["v1".to_string()]), java_catalog_props: [("jdbc.user", "admin"), ("jdbc.password", "123456")] .into_iter() @@ -1350,6 +1366,7 @@ mod test { ("s3.access.key", "hummockadmin"), ("s3.secret.key", "hummockadmin"), ("s3.region", "us-east-1"), + ("s3.path.style.access", "true"), ("catalog.name", "demo"), ("catalog.type", "storage"), ("warehouse.path", "s3://icebergdata/demo"), @@ -1374,6 +1391,7 @@ mod test { ("s3.access.key", "hummockadmin"), ("s3.secret.key", "hummockadmin"), ("s3.region", "us-east-1"), + ("s3.path.style.access", "true"), ("catalog.name", "demo"), ("catalog.type", "rest"), ("catalog.uri", "http://192.168.167.4:8181"), @@ -1399,6 +1417,7 @@ mod test { ("s3.access.key", "hummockadmin"), ("s3.secret.key", "hummockadmin"), ("s3.region", "us-east-1"), + ("s3.path.style.access", "true"), ("catalog.name", "demo"), ("catalog.type", "jdbc"), ("catalog.uri", "jdbc:postgresql://localhost:5432/iceberg"), @@ -1426,6 +1445,7 @@ mod test { ("s3.access.key", "hummockadmin"), ("s3.secret.key", "hummockadmin"), ("s3.region", "us-east-1"), + ("s3.path.style.access", "true"), ("catalog.name", "demo"), ("catalog.type", "hive"), ("catalog.uri", "thrift://localhost:9083"), diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 653acaadaaaf..36fa2559ab1a 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -272,6 +272,10 @@ IcebergConfig: - name: s3.secret.key field_type: String required: true + - name: s3.path.style.access + field_type: bool + required: false + default: Default::default - name: primary_key field_type: Vec required: false From 747245f12815715f6df6a3b97eafc1267578292d Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Tue, 20 Aug 2024 12:30:32 +0800 Subject: [PATCH 25/26] fix: udf error ui (#18118) Signed-off-by: Richard Chien --- e2e_test/error_ui/simple/main.slt | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/e2e_test/error_ui/simple/main.slt b/e2e_test/error_ui/simple/main.slt index f09e47302f3c..e6907f25fff8 100644 --- a/e2e_test/error_ui/simple/main.slt +++ b/e2e_test/error_ui/simple/main.slt @@ -27,11 +27,10 @@ db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: failed to check UDF signature 2: failed to send requests to UDF service - 3: status: Unavailable, message: "error trying to connect: tcp connect error: deadline has elapsed", details: [], metadata: MetadataMap { headers: {} } + 3: status: Unknown, message: "transport error", details: [], metadata: MetadataMap { headers: {} } 4: transport error - 5: error trying to connect - 6: tcp connect error - 7: deadline has elapsed + 5: connection error + 6: connection reset statement error From e383ad60cb3a07c0008b53921816271fd64979a1 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Tue, 20 Aug 2024 13:29:29 +0800 Subject: [PATCH 26/26] fix(connector): file source do not panic when credential is wrong (#17935) --- .../filesystem/opendal_source/opendal_enumerator.rs | 13 +++++++++++-- src/stream/src/executor/source/list_executor.rs | 4 ++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 864d1de56c7b..cffeb5dfe5f6 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -14,6 +14,7 @@ use std::marker::PhantomData; +use anyhow::anyhow; use async_trait::async_trait; use chrono::{DateTime, Utc}; use futures::stream::{self, BoxStream}; @@ -51,14 +52,22 @@ impl SplitEnumerator for OpendalEnumerator { async fn list_splits(&mut self) -> ConnectorResult>> { let empty_split: OpendalFsSplit = OpendalFsSplit::empty_split(); + let prefix = self.prefix.as_deref().unwrap_or("/"); - Ok(vec![empty_split]) + match self.op.list(prefix).await { + Ok(_) => return Ok(vec![empty_split]), + Err(e) => { + return Err(anyhow!(e) + .context("fail to create source, please check your config.") + .into()) + } + } } } impl OpendalEnumerator { pub async fn list(&self) -> ConnectorResult { - let prefix = self.prefix.as_deref().unwrap_or(""); + let prefix = self.prefix.as_deref().unwrap_or("/"); let object_lister = self .op diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index 25b32c0a0e4b..c11ba773648b 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -99,6 +99,10 @@ impl FsListExecutor { .collect::>(); let res: Vec<(Op, OwnedRow)> = rows.into_iter().flatten().collect(); + if res.is_empty() { + tracing::warn!("No items were listed from source."); + return Ok(StreamChunk::default()); + } Ok(StreamChunk::from_rows( &res, &[DataType::Varchar, DataType::Timestamptz, DataType::Int64],